From 7de2e68ce90c735d92899ee491105c43f7cea771 Mon Sep 17 00:00:00 2001 From: exceptionfactory Date: Mon, 28 Oct 2024 09:25:19 -0500 Subject: [PATCH] NIFI-13938 Removed Iceberg Processors and Services This closes #9460 Signed-off-by: Joseph Witt --- nifi-assembly/pom.xml | 29 - nifi-code-coverage/pom.xml | 27 - .../nifi-iceberg-common/pom.xml | 134 -- .../iceberg/AbstractIcebergProcessor.java | 160 --- .../nifi/processors/iceberg/IcebergUtils.java | 86 -- .../iceberg/UnmatchedColumnBehavior.java | 56 - .../catalog/IcebergCatalogFactory.java | 110 -- .../catalog/IcebergJdbcClientPool.java | 40 - .../iceberg/converter/ArrayElementGetter.java | 148 --- .../iceberg/converter/DataConverter.java | 44 - .../converter/GenericDataConverters.java | 356 ------ .../converter/IcebergRecordConverter.java | 328 ----- .../iceberg/converter/RecordFieldGetter.java | 156 --- .../nifi-iceberg-processors-nar/pom.xml | 202 --- .../src/main/resources/META-INF/LICENSE | 222 ---- .../src/main/resources/META-INF/NOTICE | 505 -------- .../nifi-iceberg-processors/pom.xml | 335 ----- .../nifi/processors/iceberg/PutIceberg.java | 415 ------ .../writer/IcebergPartitionedWriter.java | 50 - .../writer/IcebergTaskWriterFactory.java | 66 - .../org.apache.nifi.processor.Processor | 16 - .../additionalDetails.md | 46 - .../iceberg/AbstractTestPutIceberg.java | 97 -- .../iceberg/TestDataFileActions.java | 206 --- .../iceberg/TestIcebergRecordConverter.java | 1124 ----------------- .../TestPutIcebergCustomValidation.java | 150 --- .../TestPutIcebergWithHadoopCatalog.java | 216 ---- .../TestPutIcebergWithHiveCatalog.java | 213 ---- .../TestPutIcebergWithJdbcCatalog.java | 171 --- .../iceberg/util/IcebergTestUtils.java | 150 --- .../src/test/resources/date.avsc | 44 - .../src/test/resources/secured-core-site.xml | 22 - .../test/resources/unsecured-core-site.xml | 22 - .../src/test/resources/user.avsc | 39 - .../nifi-iceberg-services-api-nar/pom.xml | 40 - .../src/main/resources/META-INF/LICENSE | 209 --- .../src/main/resources/META-INF/NOTICE | 501 -------- .../nifi-iceberg-services-api/pom.xml | 32 - .../iceberg/IcebergCatalogProperty.java | 43 - .../iceberg/IcebergCatalogService.java | 35 - .../services/iceberg/IcebergCatalogType.java | 24 - .../nifi-iceberg-services-nar/pom.xml | 41 - .../src/main/resources/META-INF/LICENSE | 209 --- .../src/main/resources/META-INF/NOTICE | 17 - .../nifi-iceberg-services/pom.xml | 50 - .../iceberg/AbstractCatalogService.java | 101 -- .../iceberg/HadoopCatalogService.java | 55 - .../services/iceberg/HiveCatalogService.java | 139 -- .../services/iceberg/JdbcCatalogService.java | 78 -- ...g.apache.nifi.controller.ControllerService | 18 - .../nifi-iceberg-test-utils/pom.xml | 173 --- .../nifi/hive/metastore/MetastoreCore.java | 204 --- .../nifi/hive/metastore/ScriptRunner.java | 79 -- .../nifi/hive/metastore/ThriftMetastore.java | 75 -- .../hive-schema-4.0.0-alpha-2.derby.sql | 791 ------------ .../nifi-iceberg-bundle/pom.xml | 173 --- .../src/main/resources/META-INF/NOTICE | 2 +- nifi-extension-bundles/pom.xml | 1 - 58 files changed, 1 insertion(+), 9074 deletions(-) delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/AbstractIcebergProcessor.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/IcebergUtils.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/UnmatchedColumnBehavior.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergCatalogFactory.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergJdbcClientPool.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/ArrayElementGetter.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/DataConverter.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/GenericDataConverters.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/IcebergRecordConverter.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/RecordFieldGetter.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/LICENSE delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/NOTICE delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/PutIceberg.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergPartitionedWriter.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergTaskWriterFactory.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/docs/org.apache.nifi.processors.iceberg.PutIceberg/additionalDetails.md delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/AbstractTestPutIceberg.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestDataFileActions.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestIcebergRecordConverter.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergCustomValidation.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHadoopCatalog.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHiveCatalog.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithJdbcCatalog.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/util/IcebergTestUtils.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/date.avsc delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/secured-core-site.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/unsecured-core-site.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/user.avsc delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/LICENSE delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/NOTICE delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogProperty.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogService.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogType.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/LICENSE delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/NOTICE delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/AbstractCatalogService.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HadoopCatalogService.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HiveCatalogService.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/JdbcCatalogService.java delete mode 100755 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/pom.xml delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/MetastoreCore.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ScriptRunner.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ThriftMetastore.java delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/resources/hive-schema-4.0.0-alpha-2.derby.sql delete mode 100644 nifi-extension-bundles/nifi-iceberg-bundle/pom.xml diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 5470ec781a..ce3874827e 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -1154,35 +1154,6 @@ language governing permissions and limitations under the License. --> - - include-iceberg - - false - - allProfiles - - - - - org.apache.nifi - nifi-iceberg-processors-nar - 2.0.0-SNAPSHOT - nar - - - org.apache.nifi - nifi-iceberg-services-api-nar - 2.0.0-SNAPSHOT - nar - - - org.apache.nifi - nifi-iceberg-services-nar - 2.0.0-SNAPSHOT - nar - - - include-all diff --git a/nifi-code-coverage/pom.xml b/nifi-code-coverage/pom.xml index d5d91c8af8..6b845b066b 100644 --- a/nifi-code-coverage/pom.xml +++ b/nifi-code-coverage/pom.xml @@ -1000,33 +1000,6 @@ nifi-hubspot-processors 2.0.0-SNAPSHOT - - org.apache.nifi - nifi-iceberg-common - 2.0.0-SNAPSHOT - - - - org.bouncycastle - bcprov-jdk15on - - - - - org.apache.nifi - nifi-iceberg-processors - 2.0.0-SNAPSHOT - - - org.apache.nifi - nifi-iceberg-services - 2.0.0-SNAPSHOT - - - org.apache.nifi - nifi-iceberg-services-api - 2.0.0-SNAPSHOT - org.apache.nifi nifi-iotdb-processors diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/pom.xml deleted file mode 100644 index 232068391b..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/pom.xml +++ /dev/null @@ -1,134 +0,0 @@ - - - - 4.0.0 - - org.apache.nifi - nifi-iceberg-bundle - 2.0.0-SNAPSHOT - - - nifi-iceberg-common - jar - - - - - - org.apache.nifi - nifi-record - - - org.apache.nifi - nifi-utils - - - org.apache.nifi - nifi-record-serialization-service-api - - - org.apache.nifi - nifi-iceberg-services-api - 2.0.0-SNAPSHOT - provided - - - org.apache.nifi - nifi-kerberos-user-service-api - - - org.apache.nifi - nifi-security-kerberos-api - - - org.apache.nifi - nifi-hadoop-utils - 2.0.0-SNAPSHOT - - - org.apache.nifi - nifi-dbcp-service-api - - - - - org.apache.iceberg - iceberg-core - ${iceberg.version} - - - org.apache.iceberg - iceberg-hive-metastore - ${iceberg.version} - - - org.apache.iceberg - iceberg-data - ${iceberg.version} - - - org.apache.iceberg - iceberg-parquet - ${iceberg.version} - - - org.apache.iceberg - iceberg-orc - ${iceberg.version} - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.slf4j - slf4j-reload4j - - - commons-logging - commons-logging - - - javax.servlet - javax.servlet-api - - - - org.eclipse.jetty.websocket - websocket-client - - - org.eclipse.jetty - jetty-util - - - - - commons-lang - commons-lang - 2.6 - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/AbstractIcebergProcessor.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/AbstractIcebergProcessor.java deleted file mode 100644 index fdea6b07d4..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/AbstractIcebergProcessor.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading; -import org.apache.nifi.annotation.lifecycle.OnScheduled; -import org.apache.nifi.components.ClassloaderIsolationKeyProvider; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.context.PropertyContext; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.hadoop.SecurityUtil; -import org.apache.nifi.kerberos.KerberosUserService; -import org.apache.nifi.processor.AbstractProcessor; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.security.krb.KerberosUser; -import org.apache.nifi.services.iceberg.IcebergCatalogService; -import org.ietf.jgss.GSSException; - -import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; - -import static org.apache.nifi.hadoop.SecurityUtil.getUgiForKerberosUser; -import static org.apache.nifi.processors.iceberg.IcebergUtils.findCause; -import static org.apache.nifi.processors.iceberg.IcebergUtils.getConfigurationFromFiles; - -/** - * Base Iceberg processor class. - */ -@RequiresInstanceClassLoading(cloneAncestorResources = true) -public abstract class AbstractIcebergProcessor extends AbstractProcessor implements ClassloaderIsolationKeyProvider { - - public static final PropertyDescriptor CATALOG = new PropertyDescriptor.Builder() - .name("catalog-service") - .displayName("Catalog Service") - .description("Specifies the Controller Service to use for handling references to table’s metadata files.") - .identifiesControllerService(IcebergCatalogService.class) - .required(true) - .build(); - - public static final PropertyDescriptor KERBEROS_USER_SERVICE = new PropertyDescriptor.Builder() - .name("kerberos-user-service") - .displayName("Kerberos User Service") - .description("Specifies the Kerberos User Controller Service that should be used for authenticating with Kerberos.") - .identifiesControllerService(KerberosUserService.class) - .build(); - - public static final Relationship REL_FAILURE = new Relationship.Builder() - .name("failure") - .description("A FlowFile is routed to this relationship if the operation failed and retrying the operation will also fail, such as an invalid data or schema.") - .build(); - - protected static final AtomicReference kerberosUserReference = new AtomicReference<>(); - protected final AtomicReference ugiReference = new AtomicReference<>(); - - @OnScheduled - public void onScheduled(final ProcessContext context) { - initKerberosCredentials(context); - } - - protected synchronized void initKerberosCredentials(ProcessContext context) { - final KerberosUserService kerberosUserService = context.getProperty(KERBEROS_USER_SERVICE).asControllerService(KerberosUserService.class); - final IcebergCatalogService catalogService = context.getProperty(CATALOG).asControllerService(IcebergCatalogService.class); - - if (kerberosUserService != null) { - KerberosUser kerberosUser; - if (kerberosUserReference.get() == null) { - kerberosUser = kerberosUserService.createKerberosUser(); - } else { - kerberosUser = kerberosUserReference.get(); - } - try { - ugiReference.set(getUgiForKerberosUser(getConfigurationFromFiles(catalogService.getConfigFilePaths()), kerberosUser)); - } catch (IOException e) { - throw new ProcessException("Kerberos authentication failed", e); - } - kerberosUserReference.set(kerberosUser); - } - } - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - final FlowFile flowFile = session.get(); - if (flowFile == null) { - return; - } - - final KerberosUser kerberosUser = kerberosUserReference.get(); - if (kerberosUser == null) { - doOnTrigger(context, session, flowFile); - } else { - try { - getUgi().doAs((PrivilegedExceptionAction) () -> { - doOnTrigger(context, session, flowFile); - return null; - }); - - } catch (Exception e) { - if (!handleAuthErrors(e, session, context)) { - getLogger().error("Privileged action failed with kerberos user " + kerberosUser, e); - session.transfer(session.penalize(flowFile), REL_FAILURE); - } - } - } - } - - @Override - public String getClassloaderIsolationKey(PropertyContext context) { - try { - final KerberosUserService kerberosUserService = context.getProperty(KERBEROS_USER_SERVICE).asControllerService(KerberosUserService.class); - if (kerberosUserService != null) { - final KerberosUser kerberosUser = kerberosUserService.createKerberosUser(); - return kerberosUser.getPrincipal(); - } - } catch (IllegalStateException e) { - // the Kerberos controller service is disabled, therefore this part of the isolation key cannot be determined yet - } - - return null; - } - - private UserGroupInformation getUgi() { - SecurityUtil.checkTGTAndRelogin(getLogger(), kerberosUserReference.get()); - return ugiReference.get(); - } - - protected boolean handleAuthErrors(Throwable t, ProcessSession session, ProcessContext context) { - final Optional causeOptional = findCause(t, GSSException.class, gsse -> GSSException.NO_CRED == gsse.getMajor()); - if (causeOptional.isPresent()) { - getLogger().info("No valid Kerberos credential found, retrying login", causeOptional.get()); - kerberosUserReference.get().checkTGTAndRelogin(); - session.rollback(); - context.yield(); - return true; - } - return false; - } - - protected abstract void doOnTrigger(ProcessContext context, ProcessSession session, FlowFile flowFile) throws ProcessException; -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/IcebergUtils.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/IcebergUtils.java deleted file mode 100644 index ba2680c37e..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/IcebergUtils.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import com.google.common.base.Throwables; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.ProcessContext; - -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.Predicate; -import java.util.stream.Collectors; - -public class IcebergUtils { - - /** - * Loads configuration files from the provided paths. - * - * @param configFilePaths list of config file paths separated with comma - * @return merged configuration - */ - public static Configuration getConfigurationFromFiles(List configFilePaths) { - final Configuration conf = new Configuration(); - if (configFilePaths != null) { - for (final String configFile : configFilePaths) { - conf.addResource(new Path(configFile.trim())); - } - } - return conf; - } - - /** - * Collects every non-blank dynamic property from the context. - * - * @param context process context - * @param flowFile FlowFile to evaluate attribute expressions - * @return Map of dynamic properties - */ - public static Map getDynamicProperties(ProcessContext context, FlowFile flowFile) { - return context.getProperties().entrySet().stream() - // filter non-blank dynamic properties - .filter(e -> e.getKey().isDynamic() - && StringUtils.isNotBlank(e.getValue()) - && StringUtils.isNotBlank(context.getProperty(e.getKey()).evaluateAttributeExpressions(flowFile).getValue()) - ) - // convert to Map keys and evaluated property values - .collect(Collectors.toMap( - e -> e.getKey().getName(), - e -> context.getProperty(e.getKey()).evaluateAttributeExpressions(flowFile).getValue() - )); - } - - /** - * Returns an optional with the first throwable in the causal chain that is assignable to the provided cause type, - * and satisfies the provided cause predicate, {@link Optional#empty()} otherwise. - * - * @param t The throwable to inspect for the cause. - * @return Throwable Cause - */ - public static Optional findCause(Throwable t, Class expectedCauseType, Predicate causePredicate) { - return Throwables.getCausalChain(t).stream() - .filter(expectedCauseType::isInstance) - .map(expectedCauseType::cast) - .filter(causePredicate) - .findFirst(); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/UnmatchedColumnBehavior.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/UnmatchedColumnBehavior.java deleted file mode 100644 index 094f0daf60..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/UnmatchedColumnBehavior.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.nifi.components.DescribedValue; - -public enum UnmatchedColumnBehavior implements DescribedValue { - IGNORE_UNMATCHED_COLUMN("Ignore Unmatched Columns", - "Any column in the database that does not have a field in the document will be assumed to not be required. No notification will be logged"), - - WARNING_UNMATCHED_COLUMN("Warn on Unmatched Columns", - "Any column in the database that does not have a field in the document will be assumed to not be required. A warning will be logged"), - - FAIL_UNMATCHED_COLUMN("Fail on Unmatched Columns", - "A flow will fail if any column in the database that does not have a field in the document. An error will be logged"); - - - private final String displayName; - private final String description; - - UnmatchedColumnBehavior(final String displayName, final String description) { - this.displayName = displayName; - this.description = description; - } - - @Override - public String getValue() { - return name(); - } - - @Override - public String getDisplayName() { - return displayName; - } - - @Override - public String getDescription() { - return description; - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergCatalogFactory.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergCatalogFactory.java deleted file mode 100644 index eac1981735..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergCatalogFactory.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.catalog; - -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.jdbc.JdbcCatalog; -import org.apache.iceberg.jdbc.JdbcClientPool; -import org.apache.nifi.dbcp.DBCPService; -import org.apache.nifi.services.iceberg.IcebergCatalogProperty; -import org.apache.nifi.services.iceberg.IcebergCatalogService; - -import java.util.HashMap; -import java.util.Map; -import java.util.function.Function; - -import static org.apache.nifi.processors.iceberg.IcebergUtils.getConfigurationFromFiles; -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.CATALOG_NAME; -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.CLIENT_POOL_SERVICE; -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.METASTORE_URI; -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.WAREHOUSE_LOCATION; - -public class IcebergCatalogFactory { - - private final IcebergCatalogService catalogService; - - public IcebergCatalogFactory(IcebergCatalogService catalogService) { - this.catalogService = catalogService; - } - - public Catalog create() { - return switch (catalogService.getCatalogType()) { - case HIVE -> initHiveCatalog(catalogService); - case HADOOP -> initHadoopCatalog(catalogService); - case JDBC -> initJdbcCatalog(catalogService); - }; - } - - private Catalog initHiveCatalog(IcebergCatalogService catalogService) { - HiveCatalog catalog = new HiveCatalog(); - - if (catalogService.getConfigFilePaths() != null) { - final Configuration configuration = getConfigurationFromFiles(catalogService.getConfigFilePaths()); - catalog.setConf(configuration); - } - - final Map catalogProperties = catalogService.getCatalogProperties(); - final Map properties = new HashMap<>(); - - if (catalogProperties.containsKey(METASTORE_URI)) { - properties.put(CatalogProperties.URI, (String) catalogProperties.get(METASTORE_URI)); - } - - if (catalogProperties.containsKey(WAREHOUSE_LOCATION)) { - properties.put(CatalogProperties.WAREHOUSE_LOCATION, (String) catalogProperties.get(WAREHOUSE_LOCATION)); - } - - catalog.initialize("hive-catalog", properties); - return catalog; - } - - private Catalog initHadoopCatalog(IcebergCatalogService catalogService) { - final Map catalogProperties = catalogService.getCatalogProperties(); - final String warehousePath = (String) catalogProperties.get(WAREHOUSE_LOCATION); - - if (catalogService.getConfigFilePaths() != null) { - return new HadoopCatalog(getConfigurationFromFiles(catalogService.getConfigFilePaths()), warehousePath); - } else { - return new HadoopCatalog(new Configuration(), warehousePath); - } - } - - private Catalog initJdbcCatalog(IcebergCatalogService catalogService) { - final Map catalogProperties = catalogService.getCatalogProperties(); - final Map properties = new HashMap<>(); - properties.put(CatalogProperties.URI, ""); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, (String) catalogProperties.get(WAREHOUSE_LOCATION)); - - final Configuration configuration = getConfigurationFromFiles(catalogService.getConfigFilePaths()); - final DBCPService dbcpService = (DBCPService) catalogProperties.get(CLIENT_POOL_SERVICE); - - final Function, JdbcClientPool> clientPoolBuilder = props -> new IcebergJdbcClientPool(props, dbcpService); - final Function, FileIO> ioBuilder = props -> CatalogUtil.loadFileIO("org.apache.iceberg.hadoop.HadoopFileIO", props, configuration); - - JdbcCatalog catalog = new JdbcCatalog(ioBuilder, clientPoolBuilder, false); - catalog.setConf(configuration); - catalog.initialize((String) catalogProperties.get(CATALOG_NAME), properties); - return catalog; - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergJdbcClientPool.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergJdbcClientPool.java deleted file mode 100644 index 1b3fa0f673..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/catalog/IcebergJdbcClientPool.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.catalog; - -import org.apache.iceberg.jdbc.JdbcClientPool; -import org.apache.nifi.dbcp.DBCPService; - -import java.sql.Connection; -import java.util.Map; - -public class IcebergJdbcClientPool extends JdbcClientPool { - - private final DBCPService dbcpService; - - public IcebergJdbcClientPool(Map properties, DBCPService dbcpService) { - super("", properties); - this.dbcpService = dbcpService; - } - - @Override - protected Connection newClient() { - return dbcpService.getConnection(); - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/ArrayElementGetter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/ArrayElementGetter.java deleted file mode 100644 index 7e0be786e7..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/ArrayElementGetter.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.converter; - -import org.apache.nifi.serialization.record.DataType; -import org.apache.nifi.serialization.record.field.FieldConverter; -import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry; -import org.apache.nifi.serialization.record.type.ArrayDataType; -import org.apache.nifi.serialization.record.type.ChoiceDataType; -import org.apache.nifi.serialization.record.type.EnumDataType; -import org.apache.nifi.serialization.record.util.DataTypeUtils; -import org.apache.nifi.serialization.record.util.IllegalTypeConversionException; - -import javax.annotation.Nullable; -import java.io.Serializable; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.util.Optional; - -public class ArrayElementGetter { - - private static final String ARRAY_FIELD_NAME = "array element"; - - /** - * Creates an accessor for getting elements in an internal array data structure at the given - * position. - * - * @param dataType the element type of the array - */ - public static ElementGetter createElementGetter(DataType dataType) { - ElementGetter elementGetter; - switch (dataType.getFieldType()) { - case STRING: - elementGetter = element -> DataTypeUtils.toString(element, ARRAY_FIELD_NAME); - break; - case CHAR: - elementGetter = element -> DataTypeUtils.toCharacter(element, ARRAY_FIELD_NAME); - break; - case BOOLEAN: - elementGetter = element -> DataTypeUtils.toBoolean(element, ARRAY_FIELD_NAME); - break; - case DECIMAL: - elementGetter = element -> DataTypeUtils.toBigDecimal(element, ARRAY_FIELD_NAME); - break; - case BYTE: - elementGetter = element -> DataTypeUtils.toByte(element, ARRAY_FIELD_NAME); - break; - case SHORT: - elementGetter = element -> DataTypeUtils.toShort(element, ARRAY_FIELD_NAME); - break; - case INT: - elementGetter = element -> DataTypeUtils.toInteger(element, ARRAY_FIELD_NAME); - break; - case DATE: - elementGetter = element -> { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class); - return converter.convertField(element, Optional.ofNullable(dataType.getFormat()), ARRAY_FIELD_NAME); - }; - break; - case TIME: - elementGetter = element -> { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Time.class); - return converter.convertField(element, Optional.ofNullable(dataType.getFormat()), ARRAY_FIELD_NAME); - }; - break; - case LONG: - elementGetter = element -> DataTypeUtils.toLong(element, ARRAY_FIELD_NAME); - break; - case BIGINT: - elementGetter = element -> DataTypeUtils.toBigInt(element, ARRAY_FIELD_NAME); - break; - case FLOAT: - elementGetter = element -> DataTypeUtils.toFloat(element, ARRAY_FIELD_NAME); - break; - case DOUBLE: - elementGetter = element -> DataTypeUtils.toDouble(element, ARRAY_FIELD_NAME); - break; - case TIMESTAMP: - elementGetter = element -> { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class); - return converter.convertField(element, Optional.ofNullable(dataType.getFormat()), ARRAY_FIELD_NAME); - }; - break; - case ENUM: - elementGetter = element -> DataTypeUtils.toEnum(element, (EnumDataType) dataType, ARRAY_FIELD_NAME); - break; - case UUID: - elementGetter = DataTypeUtils::toUUID; - break; - case ARRAY: - elementGetter = element -> DataTypeUtils.toArray(element, ARRAY_FIELD_NAME, ((ArrayDataType) dataType).getElementType()); - break; - case MAP: - elementGetter = element -> DataTypeUtils.toMap(element, ARRAY_FIELD_NAME); - break; - case RECORD: - elementGetter = element -> DataTypeUtils.toRecord(element, ARRAY_FIELD_NAME); - break; - case CHOICE: - elementGetter = element -> { - final ChoiceDataType choiceDataType = (ChoiceDataType) dataType; - final DataType chosenDataType = DataTypeUtils.chooseDataType(element, choiceDataType); - if (chosenDataType == null) { - throw new IllegalTypeConversionException(String.format( - "Cannot convert value [%s] of type %s for array element to any of the following available Sub-Types for a Choice: %s", - element, element.getClass(), choiceDataType.getPossibleSubTypes())); - } - - return DataTypeUtils.convertType(element, chosenDataType, ARRAY_FIELD_NAME); - }; - break; - default: - throw new IllegalArgumentException("Unsupported field type: " + dataType.getFieldType()); - } - - return element -> { - if (element == null) { - return null; - } - - return elementGetter.getElementOrNull(element); - }; - } - - /** - * Accessor for getting the elements of an array during runtime. - */ - public interface ElementGetter extends Serializable { - @Nullable - Object getElementOrNull(Object element); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/DataConverter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/DataConverter.java deleted file mode 100644 index da549b263a..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/DataConverter.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.converter; - -/** - * Interface for data conversion between NiFi Record and Iceberg Record. - */ -public abstract class DataConverter { - - private String sourceFieldName; - private String targetFieldName; - - public String getSourceFieldName() { - return sourceFieldName; - } - - public String getTargetFieldName() { - return targetFieldName; - } - - public void setSourceFieldName(String sourceFieldName) { - this.sourceFieldName = sourceFieldName; - } - - public void setTargetFieldName(String targetFieldName) { - this.targetFieldName = targetFieldName; - } - - abstract T convert(S data); -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/GenericDataConverters.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/GenericDataConverters.java deleted file mode 100644 index e11041c4fc..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/GenericDataConverters.java +++ /dev/null @@ -1,356 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.converter; - -import org.apache.commons.lang3.ArrayUtils; -import org.apache.commons.lang3.Validate; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.nifi.serialization.SimpleRecordSchema; -import org.apache.nifi.serialization.record.DataType; -import org.apache.nifi.serialization.record.Record; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordFieldType; -import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.serialization.record.field.FieldConverter; -import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry; -import org.apache.nifi.serialization.record.util.DataTypeUtils; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.UUID; - -import static org.apache.nifi.processors.iceberg.converter.RecordFieldGetter.createFieldGetter; - -/** - * Data converter implementations for different data types. - */ -public class GenericDataConverters { - - static class PrimitiveTypeConverter extends DataConverter { - final Type.PrimitiveType targetType; - final DataType sourceType; - - public PrimitiveTypeConverter(final Type.PrimitiveType type, final DataType dataType) { - targetType = type; - sourceType = dataType; - } - - @Override - public Object convert(Object data) { - switch (targetType.typeId()) { - case BOOLEAN: - return DataTypeUtils.toBoolean(data, null); - case INTEGER: - return DataTypeUtils.toInteger(data, null); - case LONG: - return DataTypeUtils.toLong(data, null); - case FLOAT: - return DataTypeUtils.toFloat(data, null); - case DOUBLE: - return DataTypeUtils.toDouble(data, null); - case DATE: - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class); - return converter.convertField(data, Optional.ofNullable(sourceType.getFormat()), null); - case UUID: - return DataTypeUtils.toUUID(data); - case STRING: - default: - return StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class).convertField(data, Optional.empty(), null); - } - } - } - - static class TimeConverter extends DataConverter { - - private final String timeFormat; - - public TimeConverter(final String format) { - this.timeFormat = format; - } - - @Override - public LocalTime convert(Object data) { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalTime.class); - return converter.convertField(data, Optional.ofNullable(timeFormat), null); - } - } - - static class TimestampConverter extends DataConverter { - - private final DataType dataType; - - public TimestampConverter(final DataType dataType) { - this.dataType = dataType; - } - - @Override - public LocalDateTime convert(Object data) { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDateTime.class); - return converter.convertField(data, Optional.ofNullable(dataType.getFormat()), null); - } - } - - static class TimestampWithTimezoneConverter extends DataConverter { - - private final DataType dataType; - - public TimestampWithTimezoneConverter(final DataType dataType) { - this.dataType = dataType; - } - - @Override - public OffsetDateTime convert(Object data) { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(OffsetDateTime.class); - return converter.convertField(data, Optional.ofNullable(dataType.getFormat()), null); - } - } - - static class UUIDtoByteArrayConverter extends DataConverter { - - @Override - public byte[] convert(Object data) { - if (data == null) { - return null; - } - final UUID uuid = DataTypeUtils.toUUID(data); - ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[16]); - byteBuffer.putLong(uuid.getMostSignificantBits()); - byteBuffer.putLong(uuid.getLeastSignificantBits()); - return byteBuffer.array(); - } - } - - static class FixedConverter extends DataConverter { - - private final int length; - - FixedConverter(int length) { - this.length = length; - } - - @Override - public byte[] convert(Byte[] data) { - if (data == null) { - return null; - } - Validate.isTrue(data.length == length, String.format("Cannot write byte array of length %s as fixed[%s]", data.length, length)); - return ArrayUtils.toPrimitive(data); - } - } - - static class BinaryConverter extends DataConverter { - - @Override - public ByteBuffer convert(Byte[] data) { - if (data == null) { - return null; - } - return ByteBuffer.wrap(ArrayUtils.toPrimitive(data)); - } - } - - static class BigDecimalConverter extends DataConverter { - private final int precision; - private final int scale; - - BigDecimalConverter(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public BigDecimal convert(Object data) { - if (data == null) { - return null; - } - - BigDecimal bigDecimal = DataTypeUtils.toBigDecimal(data, null); - - if (bigDecimal.scale() < scale) { - bigDecimal = bigDecimal.setScale(scale); - } - - Validate.isTrue(bigDecimal.scale() == scale, "Cannot write value as decimal(%s,%s), wrong scale %s for value: %s", precision, scale, bigDecimal.scale(), data); - Validate.isTrue(bigDecimal.precision() <= precision, "Cannot write value as decimal(%s,%s), invalid precision %s for value: %s", - precision, scale, bigDecimal.precision(), data); - return bigDecimal; - } - } - - static class ArrayConverter extends DataConverter> { - private final DataConverter fieldConverter; - private final ArrayElementGetter.ElementGetter elementGetter; - - ArrayConverter(DataConverter elementConverter, DataType dataType) { - this.fieldConverter = elementConverter; - this.elementGetter = ArrayElementGetter.createElementGetter(dataType); - } - - @Override - @SuppressWarnings("unchecked") - public List convert(S[] data) { - if (data == null) { - return null; - } - final int numElements = data.length; - final List result = new ArrayList<>(numElements); - for (int i = 0; i < numElements; i += 1) { - result.add(i, fieldConverter.convert((S) elementGetter.getElementOrNull(data[i]))); - } - return result; - } - } - - static class MapConverter extends DataConverter, Map> { - private final DataConverter keyConverter; - private final DataConverter valueConverter; - private final ArrayElementGetter.ElementGetter keyGetter; - private final ArrayElementGetter.ElementGetter valueGetter; - - MapConverter(DataConverter keyConverter, DataType keyType, DataConverter valueConverter, DataType valueType) { - this.keyConverter = keyConverter; - this.keyGetter = ArrayElementGetter.createElementGetter(keyType); - this.valueConverter = valueConverter; - this.valueGetter = ArrayElementGetter.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public Map convert(Map data) { - if (data == null) { - return null; - } - final int mapSize = data.size(); - final Object[] keyArray = data.keySet().toArray(); - final Object[] valueArray = data.values().toArray(); - final Map result = new HashMap<>(mapSize); - for (int i = 0; i < mapSize; i += 1) { - result.put(keyConverter.convert((SK) keyGetter.getElementOrNull(keyArray[i])), valueConverter.convert((SV) valueGetter.getElementOrNull(valueArray[i]))); - } - - return result; - } - } - - static class RecordConverter extends DataConverter { - - private final List> converters; - private final Map getters; - - private final Types.StructType schema; - - RecordConverter(List> converters, RecordSchema recordSchema, Types.StructType schema) { - this.schema = schema; - this.converters = converters; - this.getters = new HashMap<>(converters.size()); - - for (DataConverter converter : converters) { - final Optional recordField = recordSchema.getField(converter.getSourceFieldName()); - if (recordField.isEmpty()) { - final Types.NestedField missingField = schema.field(converter.getTargetFieldName()); - if (missingField != null) { - getters.put(converter.getTargetFieldName(), createFieldGetter(convertSchemaTypeToDataType(missingField.type()), missingField.name(), missingField.isOptional())); - } - } else { - final RecordField field = recordField.get(); - // creates a record field accessor for every data converter - getters.put(converter.getTargetFieldName(), createFieldGetter(field.getDataType(), field.getFieldName(), field.isNullable())); - } - } - } - - @Override - public GenericRecord convert(Record data) { - if (data == null) { - return null; - } - final GenericRecord record = GenericRecord.create(schema); - - for (DataConverter converter : converters) { - record.setField(converter.getTargetFieldName(), convert(data, converter)); - } - - return record; - } - - @SuppressWarnings("unchecked") - private T convert(Record record, DataConverter converter) { - return converter.convert((S) getters.get(converter.getTargetFieldName()).getFieldOrNull(record)); - } - } - - public static DataType convertSchemaTypeToDataType(Type schemaType) { - switch (schemaType.typeId()) { - case BOOLEAN: - return RecordFieldType.BOOLEAN.getDataType(); - case INTEGER: - return RecordFieldType.INT.getDataType(); - case LONG: - return RecordFieldType.LONG.getDataType(); - case FLOAT: - return RecordFieldType.FLOAT.getDataType(); - case DOUBLE: - return RecordFieldType.DOUBLE.getDataType(); - case DATE: - return RecordFieldType.DATE.getDataType(); - case TIME: - return RecordFieldType.TIME.getDataType(); - case TIMESTAMP: - return RecordFieldType.TIMESTAMP.getDataType(); - case STRING: - return RecordFieldType.STRING.getDataType(); - case UUID: - return RecordFieldType.UUID.getDataType(); - case FIXED: - case BINARY: - return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()); - case DECIMAL: - return RecordFieldType.DECIMAL.getDataType(); - case STRUCT: - // Build a record type from the struct type - Types.StructType structType = schemaType.asStructType(); - List fields = structType.fields(); - List recordFields = new ArrayList<>(fields.size()); - for (Types.NestedField field : fields) { - DataType dataType = convertSchemaTypeToDataType(field.type()); - recordFields.add(new RecordField(field.name(), dataType, field.isOptional())); - } - RecordSchema recordSchema = new SimpleRecordSchema(recordFields); - return RecordFieldType.RECORD.getRecordDataType(recordSchema); - case LIST: - // Build a list type from the elements - Types.ListType listType = schemaType.asListType(); - return RecordFieldType.ARRAY.getArrayDataType(convertSchemaTypeToDataType(listType.elementType()), listType.isElementOptional()); - case MAP: - // Build a map type from the elements - Types.MapType mapType = schemaType.asMapType(); - return RecordFieldType.MAP.getMapDataType(convertSchemaTypeToDataType(mapType.valueType()), mapType.isValueOptional()); - } - throw new IllegalArgumentException("Invalid or unsupported type: " + schemaType); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/IcebergRecordConverter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/IcebergRecordConverter.java deleted file mode 100644 index d4b504f3e4..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/IcebergRecordConverter.java +++ /dev/null @@ -1,328 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.nifi.processors.iceberg.converter; - -import org.apache.commons.lang.Validate; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.schema.SchemaWithPartnerVisitor; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processors.iceberg.UnmatchedColumnBehavior; -import org.apache.nifi.serialization.record.DataType; -import org.apache.nifi.serialization.record.Record; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordFieldType; -import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.serialization.record.type.ArrayDataType; -import org.apache.nifi.serialization.record.type.MapDataType; -import org.apache.nifi.serialization.record.type.RecordDataType; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - -/** - * This class is responsible for schema traversal and data conversion between NiFi and Iceberg internal record structure. - */ -public class IcebergRecordConverter { - - private final DataConverter converter; - public final UnmatchedColumnBehavior unmatchedColumnBehavior; - public ComponentLog logger; - - public GenericRecord convert(Record record) { - return converter.convert(record); - } - - - @SuppressWarnings("unchecked") - public IcebergRecordConverter(Schema schema, RecordSchema recordSchema, FileFormat fileFormat, UnmatchedColumnBehavior unmatchedColumnBehavior, ComponentLog logger) { - this.converter = (DataConverter) IcebergSchemaVisitor.visit(schema, new RecordDataType(recordSchema), fileFormat, unmatchedColumnBehavior, logger); - this.unmatchedColumnBehavior = unmatchedColumnBehavior; - this.logger = logger; - } - - private static class IcebergSchemaVisitor extends SchemaWithPartnerVisitor> { - - public static DataConverter visit(Schema schema, RecordDataType recordDataType, FileFormat fileFormat, UnmatchedColumnBehavior unmatchedColumnBehavior, ComponentLog logger) { - return visit(schema, new RecordTypeWithFieldNameMapper(schema, recordDataType), new IcebergSchemaVisitor(), - new IcebergPartnerAccessors(schema, fileFormat, unmatchedColumnBehavior, logger)); - } - - @Override - public DataConverter schema(Schema schema, DataType dataType, DataConverter converter) { - return converter; - } - - @Override - public DataConverter field(Types.NestedField field, DataType dataType, DataConverter converter) { - // set Iceberg schema field name (targetFieldName) in the data converter - converter.setTargetFieldName(field.name()); - return converter; - } - - @Override - public DataConverter primitive(Type.PrimitiveType type, DataType dataType) { - if (type.typeId() != null) { - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - case DATE: - case STRING: - return new GenericDataConverters.PrimitiveTypeConverter(type, dataType); - case TIME: - return new GenericDataConverters.TimeConverter(dataType.getFormat()); - case TIMESTAMP: - final Types.TimestampType timestampType = (Types.TimestampType) type; - if (timestampType.shouldAdjustToUTC()) { - return new GenericDataConverters.TimestampWithTimezoneConverter(dataType); - } - return new GenericDataConverters.TimestampConverter(dataType); - case UUID: - final UUIDDataType uuidType = (UUIDDataType) dataType; - if (uuidType.getFileFormat() == FileFormat.PARQUET) { - return new GenericDataConverters.UUIDtoByteArrayConverter(); - } - return new GenericDataConverters.PrimitiveTypeConverter(type, dataType); - case FIXED: - final Types.FixedType fixedType = (Types.FixedType) type; - return new GenericDataConverters.FixedConverter(fixedType.length()); - case BINARY: - return new GenericDataConverters.BinaryConverter(); - case DECIMAL: - final Types.DecimalType decimalType = (Types.DecimalType) type; - return new GenericDataConverters.BigDecimalConverter(decimalType.precision(), decimalType.scale()); - default: - throw new UnsupportedOperationException("Unsupported type: " + type.typeId()); - } - } - throw new UnsupportedOperationException("Missing type id from PrimitiveType " + type); - } - - @Override - public DataConverter struct(Types.StructType type, DataType dataType, List> converters) { - Validate.notNull(type, "Can not create reader for null type"); - final RecordTypeWithFieldNameMapper recordType = (RecordTypeWithFieldNameMapper) dataType; - final RecordSchema recordSchema = recordType.getChildSchema(); - - // set NiFi schema field names (sourceFieldName) in the data converters - for (DataConverter converter : converters) { - final Optional mappedFieldName = recordType.getNameMapping(converter.getTargetFieldName()); - if (mappedFieldName.isPresent()) { - final Optional recordField = recordSchema.getField(mappedFieldName.get()); - converter.setSourceFieldName(recordField.get().getFieldName()); - } - } - - return new GenericDataConverters.RecordConverter(converters, recordSchema, type); - } - - @Override - public DataConverter list(Types.ListType listTypeInfo, DataType dataType, DataConverter converter) { - return new GenericDataConverters.ArrayConverter<>(converter, ((ArrayDataType) dataType).getElementType()); - } - - @Override - public DataConverter map(Types.MapType mapType, DataType dataType, DataConverter keyConverter, DataConverter valueConverter) { - return new GenericDataConverters.MapConverter<>(keyConverter, RecordFieldType.STRING.getDataType(), valueConverter, ((MapDataType) dataType).getValueType()); - } - } - - public static class IcebergPartnerAccessors implements SchemaWithPartnerVisitor.PartnerAccessors { - private final Schema schema; - private final FileFormat fileFormat; - private final UnmatchedColumnBehavior unmatchedColumnBehavior; - private final ComponentLog logger; - - IcebergPartnerAccessors(Schema schema, FileFormat fileFormat, UnmatchedColumnBehavior unmatchedColumnBehavior, ComponentLog logger) { - this.schema = schema; - this.fileFormat = fileFormat; - this.unmatchedColumnBehavior = unmatchedColumnBehavior; - this.logger = logger; - } - - @Override - public DataType fieldPartner(DataType dataType, int fieldId, String name) { - Validate.isTrue(dataType instanceof RecordTypeWithFieldNameMapper, String.format("Invalid record: %s is not a record", dataType)); - final RecordTypeWithFieldNameMapper recordType = (RecordTypeWithFieldNameMapper) dataType; - - final Optional mappedFieldName = recordType.getNameMapping(name); - if (UnmatchedColumnBehavior.FAIL_UNMATCHED_COLUMN.equals(unmatchedColumnBehavior)) { - Validate.isTrue(mappedFieldName.isPresent(), String.format("Cannot find field with name '%s' in the record schema", name)); - } - if (mappedFieldName.isEmpty()) { - if (UnmatchedColumnBehavior.WARNING_UNMATCHED_COLUMN.equals(unmatchedColumnBehavior)) { - if (logger != null) { - logger.warn("Cannot find field with name '{}' in the record schema, using the target schema for datatype and a null value", name); - } - } - // If the field is missing, use the expected type from the schema (converted to a DataType) - final Types.NestedField schemaField = schema.findField(fieldId); - final Type schemaFieldType = schemaField.type(); - if (schemaField.isRequired()) { - // Iceberg requires a non-null value for required fields - throw new IllegalArgumentException("Iceberg requires a non-null value for required fields, field: " - + schemaField.name() + ", type: " + schemaFieldType); - } - return GenericDataConverters.convertSchemaTypeToDataType(schemaFieldType); - } - final Optional recordField = recordType.getChildSchema().getField(mappedFieldName.get()); - final DataType fieldType = recordField.get().getDataType(); - - // If the actual record contains a nested record then we need to create a RecordTypeWithFieldNameMapper wrapper object for it. - if (fieldType instanceof RecordDataType) { - return new RecordTypeWithFieldNameMapper(new Schema(schema.findField(fieldId).type().asStructType().fields()), (RecordDataType) fieldType); - } - - // If the field is an Array, and it contains Records then add the record's iceberg schema for creating RecordTypeWithFieldNameMapper - if (fieldType instanceof ArrayDataType && ((ArrayDataType) fieldType).getElementType() instanceof RecordDataType) { - return new ArrayTypeWithIcebergSchema( - new Schema(schema.findField(fieldId).type().asListType().elementType().asStructType().fields()), - ((ArrayDataType) fieldType).getElementType() - ); - } - - // If the field is a Map, and it's value field contains Records then add the record's iceberg schema for creating RecordTypeWithFieldNameMapper - if (fieldType instanceof MapDataType && ((MapDataType) fieldType).getValueType() instanceof RecordDataType) { - return new MapTypeWithIcebergSchema( - new Schema(schema.findField(fieldId).type().asMapType().valueType().asStructType().fields()), - ((MapDataType) fieldType).getValueType() - ); - } - - // If the source field or target field is of type UUID, create a UUIDDataType from it - if (fieldType.getFieldType().equals(RecordFieldType.UUID) || schema.findField(fieldId).type().typeId() == Type.TypeID.UUID) { - return new UUIDDataType(fieldType, fileFormat); - } - - return fieldType; - } - - @Override - public DataType mapKeyPartner(DataType dataType) { - return RecordFieldType.STRING.getDataType(); - } - - @Override - public DataType mapValuePartner(DataType dataType) { - Validate.isTrue(dataType instanceof MapDataType, String.format("Invalid map: %s is not a map", dataType)); - final MapDataType mapType = (MapDataType) dataType; - if (mapType instanceof MapTypeWithIcebergSchema) { - MapTypeWithIcebergSchema typeWithSchema = (MapTypeWithIcebergSchema) mapType; - return new RecordTypeWithFieldNameMapper(typeWithSchema.getValueSchema(), (RecordDataType) typeWithSchema.getValueType()); - } - return mapType.getValueType(); - } - - @Override - public DataType listElementPartner(DataType dataType) { - Validate.isTrue(dataType instanceof ArrayDataType, String.format("Invalid array: %s is not an array", dataType)); - final ArrayDataType arrayType = (ArrayDataType) dataType; - if (arrayType instanceof ArrayTypeWithIcebergSchema) { - ArrayTypeWithIcebergSchema typeWithSchema = (ArrayTypeWithIcebergSchema) arrayType; - return new RecordTypeWithFieldNameMapper(typeWithSchema.getElementSchema(), (RecordDataType) typeWithSchema.getElementType()); - } - return arrayType.getElementType(); - } - } - - /** - * Parquet writer expects the UUID value in different format, so it needs to be converted differently: #1881 - */ - private static class UUIDDataType extends DataType { - - private final FileFormat fileFormat; - - UUIDDataType(DataType dataType, FileFormat fileFormat) { - super(dataType.getFieldType(), dataType.getFormat()); - this.fileFormat = fileFormat; - } - - public FileFormat getFileFormat() { - return fileFormat; - } - } - - /** - * Since the {@link RecordSchema} stores the field name and value pairs in a HashMap it makes the retrieval case-sensitive, so we create a name mapper for case-insensitive handling. - */ - private static class RecordTypeWithFieldNameMapper extends RecordDataType { - - private final Map fieldNameMap; - - RecordTypeWithFieldNameMapper(Schema schema, RecordDataType recordType) { - super(recordType.getChildSchema()); - - // create a lowercase map for the NiFi record schema fields - final Map lowerCaseMap = recordType.getChildSchema().getFieldNames().stream() - .collect(Collectors.toMap(String::toLowerCase, s -> s)); - - // map the Iceberg record schema fields to the NiFi record schema fields - this.fieldNameMap = new HashMap<>(); - schema.columns().forEach((s) -> this.fieldNameMap.put(s.name(), lowerCaseMap.get(s.name().toLowerCase()))); - } - - Optional getNameMapping(String name) { - return Optional.ofNullable(fieldNameMap.get(name)); - } - } - - /** - * Data type for Arrays which contains Records. The class stores the iceberg schema for the element type. - */ - private static class ArrayTypeWithIcebergSchema extends ArrayDataType { - - private final Schema elementSchema; - - public ArrayTypeWithIcebergSchema(Schema elementSchema, DataType elementType) { - super(elementType); - this.elementSchema = elementSchema; - } - - public Schema getElementSchema() { - return elementSchema; - } - } - - /** - * Data type for Maps which contains Records in the entries value. The class stores the iceberg schema for the value type. - */ - private static class MapTypeWithIcebergSchema extends MapDataType { - - private final Schema valueSchema; - - public MapTypeWithIcebergSchema(Schema valueSchema, DataType valueType) { - super(valueType); - this.valueSchema = valueSchema; - } - - public Schema getValueSchema() { - return valueSchema; - } - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/RecordFieldGetter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/RecordFieldGetter.java deleted file mode 100644 index d0f9d55d87..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-common/src/main/java/org/apache/nifi/processors/iceberg/converter/RecordFieldGetter.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.converter; - -import org.apache.nifi.serialization.record.DataType; -import org.apache.nifi.serialization.record.Record; -import org.apache.nifi.serialization.record.field.FieldConverter; -import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry; -import org.apache.nifi.serialization.record.type.ArrayDataType; -import org.apache.nifi.serialization.record.type.ChoiceDataType; -import org.apache.nifi.serialization.record.type.EnumDataType; -import org.apache.nifi.serialization.record.type.RecordDataType; -import org.apache.nifi.serialization.record.util.DataTypeUtils; -import org.apache.nifi.serialization.record.util.IllegalTypeConversionException; - -import javax.annotation.Nullable; -import java.io.Serializable; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.util.Optional; - -public class RecordFieldGetter { - - /** - * Creates an accessor for getting elements in an internal record data structure with the given - * field name. - * - * @param dataType the element type of the field - * @param fieldName the name of the field - * @param isNullable indicates if the field's value is nullable - */ - public static FieldGetter createFieldGetter(DataType dataType, String fieldName, boolean isNullable) { - FieldGetter fieldGetter; - switch (dataType.getFieldType()) { - case STRING: - fieldGetter = record -> record.getAsString(fieldName); - break; - case CHAR: - fieldGetter = record -> DataTypeUtils.toCharacter(record.getValue(fieldName), fieldName); - break; - case BOOLEAN: - fieldGetter = record -> record.getAsBoolean(fieldName); - break; - case DECIMAL: - fieldGetter = record -> DataTypeUtils.toBigDecimal(record.getValue(fieldName), fieldName); - break; - case BYTE: - fieldGetter = record -> DataTypeUtils.toByte(record.getValue(fieldName), fieldName); - break; - case SHORT: - fieldGetter = record -> DataTypeUtils.toShort(record.getValue(fieldName), fieldName); - break; - case INT: - fieldGetter = record -> record.getAsInt(fieldName); - break; - case DATE: - fieldGetter = record -> { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class); - return converter.convertField(record.getValue(fieldName), Optional.ofNullable(dataType.getFormat()), fieldName); - }; - break; - case TIME: - fieldGetter = record -> { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Time.class); - return converter.convertField(record.getValue(fieldName), Optional.ofNullable(dataType.getFormat()), fieldName); - }; - break; - case LONG: - fieldGetter = record -> record.getAsLong(fieldName); - break; - case BIGINT: - fieldGetter = record -> DataTypeUtils.toBigInt(record.getValue(fieldName), fieldName); - break; - case FLOAT: - fieldGetter = record -> record.getAsFloat(fieldName); - break; - case DOUBLE: - fieldGetter = record -> record.getAsDouble(fieldName); - break; - case TIMESTAMP: - fieldGetter = record -> { - final FieldConverter converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class); - return converter.convertField(record.getValue(fieldName), Optional.ofNullable(dataType.getFormat()), fieldName); - }; - break; - case UUID: - fieldGetter = record -> DataTypeUtils.toUUID(record.getValue(fieldName)); - break; - case ENUM: - fieldGetter = record -> DataTypeUtils.toEnum(record.getValue(fieldName), (EnumDataType) dataType, fieldName); - break; - case ARRAY: - fieldGetter = record -> DataTypeUtils.toArray(record.getValue(fieldName), fieldName, ((ArrayDataType) dataType).getElementType()); - break; - case MAP: - fieldGetter = record -> DataTypeUtils.toMap(record.getValue(fieldName), fieldName); - break; - case RECORD: - fieldGetter = record -> record.getAsRecord(fieldName, ((RecordDataType) dataType).getChildSchema()); - break; - case CHOICE: - fieldGetter = record -> { - final ChoiceDataType choiceDataType = (ChoiceDataType) dataType; - final Object value = record.getValue(fieldName); - final DataType chosenDataType = DataTypeUtils.chooseDataType(value, choiceDataType); - if (chosenDataType == null) { - throw new IllegalTypeConversionException(String.format( - "Cannot convert value [%s] of type %s for field %s to any of the following available Sub-Types for a Choice: %s", - value, value.getClass(), fieldName, choiceDataType.getPossibleSubTypes())); - } - - return DataTypeUtils.convertType(record.getValue(fieldName), chosenDataType, fieldName); - }; - break; - default: - throw new IllegalArgumentException("Unsupported field type: " + dataType.getFieldType()); - } - - if (!isNullable) { - return fieldGetter; - } - - return record -> { - if (record.getValue(fieldName) == null) { - return null; - } - - return fieldGetter.getFieldOrNull(record); - }; - } - - /** - * Accessor for getting the field of a record during runtime. - */ - - public interface FieldGetter extends Serializable { - @Nullable - Object getFieldOrNull(Record record); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/pom.xml deleted file mode 100644 index 3ec9ba5d4f..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/pom.xml +++ /dev/null @@ -1,202 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-processors-nar - nar - - - - org.apache.nifi - nifi-iceberg-processors - 2.0.0-SNAPSHOT - - - org.apache.nifi - nifi-iceberg-services-api-nar - 2.0.0-SNAPSHOT - nar - - - - - - - include-hadoop-aws - - false - - - - org.apache.hadoop - hadoop-aws - ${hadoop.version} - - - - - - include-hadoop-azure - - false - - - - org.apache.hadoop - hadoop-azure - ${hadoop.version} - - - com.google.guava - guava - - - com.fasterxml.jackson.core - jackson-core - - - commons-logging - commons-logging - - - - - org.apache.hadoop - hadoop-azure-datalake - ${hadoop.version} - - - com.fasterxml.jackson.core - jackson-core - - - - - - - - include-hadoop-cloud-storage - - false - - - - org.apache.hadoop - hadoop-cloud-storage - ${hadoop.version} - - - log4j - log4j - - - org.apache.logging.log4j - log4j-core - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - - - - - - include-hadoop-ozone - - false - - - - org.apache.ozone - ozone-client - ${ozone.version} - - - commons-logging - commons-logging - - - org.apache.logging.log4j - log4j-core - - - org.bouncycastle - bcprov-jdk15on - - - org.bouncycastle - bcpkix-jdk15on - - - - - org.apache.ozone - ozone-filesystem - ${ozone.version} - - - org.bouncycastle - bcprov-jdk18on - - - org.bouncycastle - bcpkix-jdk18on - - - - - - include-hadoop-gcp - - false - - - - com.google.cloud.bigdataoss - gcs-connector - hadoop3-${gcs.version} - - - com.google.cloud.bigdataoss - util - ${gcs.version} - - - com.google.cloud.bigdataoss - util-hadoop - hadoop3-${gcs.version} - - - com.google.cloud.bigdataoss - gcsio - ${gcs.version} - - - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/LICENSE b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/LICENSE deleted file mode 100644 index f3574b43b4..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/LICENSE +++ /dev/null @@ -1,222 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -APACHE NIFI SUBCOMPONENTS: - -The Apache NiFi project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - - --------------------------------------------------------------------------------- - -This product includes code from Apache Hive. - -* Hive metastore derby schema in hive-schema-3.2.0.derby.sql -* Test methods from IcebergTestUtils.java -* Test metastore from MetastoreCore.java, ScriptRunner.java, ThriftMetastore.java - -Copyright: 2011-2018 The Apache Software Foundation -Home page: https://hive.apache.org/ -License: https://www.apache.org/licenses/LICENSE-2.0 \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/NOTICE b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/NOTICE deleted file mode 100644 index f25c183e91..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors-nar/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,505 +0,0 @@ -nifi-iceberg-processors-nar -Copyright 2014-2024 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -=========================================== -Apache Software License v2 -=========================================== - - (ASLv2) Apache Iceberg - The following NOTICE information applies: - Apache Iceberg - Copyright 2017-2022 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - (ASLv2) Apache Ant - The following NOTICE information applies: - Apache Ant - Copyright 1999-2016 The Apache Software Foundation - - (ASLv2) Apache Commons Codec - The following NOTICE information applies: - Apache Commons Codec - Copyright 2002-2014 The Apache Software Foundation - - src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java - contains test data from http://aspell.net/test/orig/batch0.tab. - Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - - =============================================================================== - - The content of package org.apache.commons.codec.language.bm has been translated - from the original php source code available at http://stevemorse.org/phoneticinfo.htm - with permission from the original authors. - Original source copyright: - Copyright (c) 2008 Alexander Beider & Stephen P. Morse. - - (ASLv2) Apache Commons DBCP - The following NOTICE information applies: - Apache Commons DBCP - Copyright 2001-2015 The Apache Software Foundation. - - (ASLv2) Apache HttpComponents - The following NOTICE information applies: - Apache HttpComponents Client - Copyright 1999-2016 The Apache Software Foundation - Apache HttpComponents Core - HttpCore - Copyright 2006-2009 The Apache Software Foundation - - (ASLv2) Apache Commons Pool - The following NOTICE information applies: - Apache Commons Pool - Copyright 1999-2009 The Apache Software Foundation. - - (ASLv2) Apache Commons BeanUtils - The following NOTICE information applies: - Apache Commons BeanUtils - Copyright 2000-2016 The Apache Software Foundation - - (ASLv2) Apache Commons IO - The following NOTICE information applies: - Apache Commons IO - Copyright 2002-2016 The Apache Software Foundation - - (ASLv2) Apache Commons Net - The following NOTICE information applies: - Apache Commons Net - Copyright 2001-2016 The Apache Software Foundation - - (ASLv2) Apache Avro - The following NOTICE information applies: - Apache Avro - Copyright 2009-2017 The Apache Software Foundation - - (ASLv2) Apache Parquet - The following NOTICE information applies: - Apache Parquet MR (Incubating) - Copyright 2014 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - (ASLv2) Audience Annotations - The following NOTICE information applies: - Apache Yetus - Copyright 2008-2018 The Apache Software Foundation - - (ASLv2) Apache Commons Compress - The following NOTICE information applies: - Apache Commons Compress - Copyright 2002-2017 The Apache Software Foundation - - The files in the package org.apache.commons.compress.archivers.sevenz - were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), - which has been placed in the public domain: - - "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) - - (ASLv2) Apache Commons Configuration - The following NOTICE information applies: - Apache Commons Configuration - Copyright 2001-2017 The Apache Software Foundation - - (ASLv2) Apache Commons Text - The following NOTICE information applies: - Apache Commons Text - Copyright 2001-2018 The Apache Software Foundation - - (ASLv2) Apache Commons CLI - The following NOTICE information applies: - Apache Commons CLI - Copyright 2001-2017 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - (ASLv2) Apache Commons Collections - The following NOTICE information applies: - Apache Commons Collections - Copyright 2001-2016 The Apache Software Foundation - - (ASLv2) Apache Commons Daemon - The following NOTICE information applies: - Apache Commons Daemon - Copyright 2002-2013 The Apache Software Foundation - - (ASLv2) Apache Ivy - The following NOTICE information applies: - Copyright 2007-2017 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - Portions of Ivy were originally developed at - Jayasoft SARL (http://www.jayasoft.fr/) - and are licensed to the Apache Software Foundation under the - "Software Grant License Agreement" - - SSH and SFTP support is provided by the JCraft JSch package, - which is open source software, available under - the terms of a BSD style license. - The original software and related information is available - at http://www.jcraft.com/jsch/. - - (ASLv2) Apache Commons Math - The following NOTICE information applies: - Apache Commons Math - Copyright 2001-2012 The Apache Software Foundation - - (ASLv2) Apache Hive - The following NOTICE information applies: - Apache Hive - Copyright 2008-2015 The Apache Software Foundation - - This product includes software developed by The Apache Software - Foundation (http://www.apache.org/). - - This product includes Jersey (https://jersey.java.net/) - Copyright (c) 2010-2014 Oracle and/or its affiliates. - - This project includes software copyrighted by Microsoft Corporation and - licensed under the Apache License, Version 2.0. - - This project includes software copyrighted by Dell SecureWorks and - licensed under the Apache License, Version 2.0. - - (ASLv2) Jackson JSON processor - The following NOTICE information applies: - # Jackson JSON processor - - Jackson is a high-performance, Free/Open Source JSON processing library. - It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has - been in development since 2007. - It is currently developed by a community of developers, as well as supported - commercially by FasterXML.com. - - ## Licensing - - Jackson core and extension components may licensed under different licenses. - To find the details that apply to this artifact see the accompanying LICENSE file. - For more information, including possible other licensing options, contact - FasterXML.com (http://fasterxml.com). - - ## Credits - - A list of contributors may be found from CREDITS file, which is included - in some artifacts (usually source distributions); but is always available - from the source code management (SCM) system project uses. - - (ASLv2) BoneCP - The following NOTICE information applies: - BoneCP - Copyright 2010 Wallace Wadge - - (ASLv2) Apache Hadoop - The following NOTICE information applies: - The binary distribution of this product bundles binaries of - org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the - following notices: - * Copyright 2011 Dain Sundstrom - * Copyright 2011 FuseSource Corp. http://fusesource.com - - The binary distribution of this product bundles binaries of - org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni), - which has the following notices: - * This product includes software developed by FuseSource Corp. - http://fusesource.com - * This product includes software developed at - Progress Software Corporation and/or its subsidiaries or affiliates. - * This product includes software developed by IBM Corporation and others. - - (ASLv2) Apache Commons Lang - The following NOTICE information applies: - Apache Commons Lang - Copyright 2001-2015 The Apache Software Foundation - - (ASLv2) Apache Curator - The following NOTICE information applies: - Curator Framework - Copyright 2011-2014 The Apache Software Foundation - - Curator Client - Copyright 2011-2014 The Apache Software Foundation - - Curator Recipes - Copyright 2011-2014 The Apache Software Foundation - - (ASLv2) Apache Derby - The following NOTICE information applies: - Apache Derby - Copyright 2004-2014 Apache, Apache DB, Apache Derby, Apache Torque, Apache JDO, Apache DDLUtils, - the Derby hat logo, the Apache JDO logo, and the Apache feather logo are trademarks of The Apache Software Foundation. - - (ASLv2) Apache Geronimo - The following NOTICE information applies: - Apache Geronimo - Copyright 2003-2008 The Apache Software Foundation - - (ASLv2) Jettison - The following NOTICE information applies: - Copyright 2006 Envoi Solutions LLC - - (ASLv2) Jetty - The following NOTICE information applies: - Jetty Web Container - Copyright 1995-2019 Mort Bay Consulting Pty Ltd. - - (ASLv2) Apache log4j - The following NOTICE information applies: - Apache log4j - Copyright 2007 The Apache Software Foundation - - (ASLv2) Apache Thrift - The following NOTICE information applies: - Apache Thrift - Copyright 2006-2010 The Apache Software Foundation. - - (ASLv2) Dropwizard Metrics - The following NOTICE information applies: - Metrics - Copyright 2010-2013 Coda Hale and Yammer, Inc. - - This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, - LongAdder), which was released with the following comments: - - Written by Doug Lea with assistance from members of JCP JSR-166 - Expert Group and released to the public domain, as explained at - http://creativecommons.org/publicdomain/zero/1.0/ - - (ASLv2) Joda Time - The following NOTICE information applies: - This product includes software developed by - Joda.org (http://www.joda.org/). - - (ASLv2) The Netty Project - The following NOTICE information applies: - The Netty Project - Copyright 2011 The Netty Project - - (ASLv2) Apache ZooKeeper - The following NOTICE information applies: - Apache ZooKeeper - Copyright 2009-2012 The Apache Software Foundation - - (ASLv2) Google GSON - The following NOTICE information applies: - Copyright 2008 Google Inc. - - (ASLv2) JPam - The following NOTICE information applies: - Copyright 2003-2006 Greg Luck - - (ASLv2) Groovy 2.4.16 (http://www.groovy-lang.org) - groovy-2.4.16-indy - groovy-json-2.4.16-indy - groovy-sql-2.4.16-indy - The following NOTICE information applies: - Apache Groovy - Copyright 2003-2018 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - This product includes/uses ANTLR (http://www.antlr2.org/) - developed by Terence Parr 1989-2006 - - (ASLv2) ASM Based Accessors Helper Used By JSON Smart (net.minidev:accessors-smart:jar:1.2 - http://www.minidev.net/) - The following NOTICE information applies: - ASM Based Accessors Helper Used By JSON Smart 1.2 - Copyright 2017, Uriel Chemouni - - (ASLv2) JSON Smart (net.minidev:json-smart:jar:2.3 - http://www.minidev.net/) - The following NOTICE information applies: - JSON Smart 2.3 - Copyright 2017, Uriel Chemouni, Eitan Raviv - - (ASLv2) Nimbus JOSE+JWT (com.nimbusds:nimbus-jose-jwt - https://connect2id.com/products/nimbus-jose-jwt) - The following NOTICE information applies: - Nimbus JOSE+JWT - Copyright 2021, Connect2id Ltd. - - (ASLv2) Woodstox (com.fasterxml.woodstox:woodstox-core:bundle:5.3.0 - https://github.com/FasterXML/woodstox) - The following NOTICE information applies: - Woodstox Core 5.3.0 - Copyright 2015, FasterXML, LLC - - (ASLv2) Joda Time - The following NOTICE information applies: - This product includes software developed by - Joda.org (http://www.joda.org/). - - (ASLv2) java-util - The following NOTICE information applies: - java-util - Copyright 2011-2017 Metamarkets Group Inc. - - (ASLv2) JCIP Annotations Under Apache License - The following NOTICE information applies: - JCIP Annotations Under Apache License - Copyright 2013 Stephen Connolly. - - (ASLv2) Google GSON - The following NOTICE information applies: - Copyright 2008 Google Inc. - - (ASLv2) Guava - The following NOTICE information applies: - Guava - Copyright 2015 The Guava Authors - - (ASLv2) OkHttp - The following NOTICE information applies: - OkHttp - Copyright (C) 2014 Square, Inc. - - (ASLv2) Okio - The following NOTICE information applies: - Okio - Copyright (C) 2014 Square, Inc. - - (ASLv2) Dropwizard Metrics - The following NOTICE information applies: - Dropwizard Metrics - Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team - - (ASLv2) atinject (javax.inject:javax.inject) - The following NOTICE information applies: - atinject - Copyright - - (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3) - - (ASLv2) JetBrains/java-annotations - The following NOTICE information applies: - JetBrains/java-annotations - Copyright 2000-2016 JetBrains s.r.o. - - (ASLv2) Apache Kerby - The following NOTICE information applies: - Apache Kerby - Copyright 2003-2018 The Apache Software Foundation - - (ASLv2) Carrotsearch HPPC - The following NOTICE information applies: - HPPC borrowed code, ideas or both from: - - * Apache Lucene, http://lucene.apache.org/ - (Apache license) - * Fastutil, http://fastutil.di.unimi.it/ - (Apache license) - * Koloboke, https://github.com/OpenHFT/Koloboke - (Apache license) - - (ASLv2) Ehcache 2.x - The following NOTICE information applies: - Copyright 2003-2010 Terracotta, Inc. - - (ASLv2) Google Guice - The following NOTICE information applies: - Google Guice - Core Library - Copyright 2006-2011 Google, Inc. - - Google Guice - Extensions - Servlet - Copyright 2006-2011 Google, Inc. - - (ASLv2) Apache Arrow - The following NOTICE information applies: - Copyright 2016-2019 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - - (ASLv2) Apache ORC - The following NOTICE information applies: - Copyright 2013-2019 The Apache Software Foundation - - This product includes software developed by The Apache Software - Foundation (http://www.apache.org/). - - This product includes software developed by Hewlett-Packard: - (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P - -************************ -Common Development and Distribution License 1.1 -************************ - - The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details. - - (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.19 - https://jersey.java.net) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) Java Architecture For XML Binding (javax.xml.bind:jaxb-api:jar:2.2.11 - https://jaxb.dev.java.net/) - -************************ -Common Development and Distribution License 1.0 -************************ - - The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details. - - (CDDL 1.0) JavaServlet(TM) Specification (javax.servlet:servlet-api:jar:3.1.0 - no url available) - (CDDL 1.0) JavaServer Pages(TM) API (javax.servlet.jsp:jsp-api:jar:2.1 - http://jsp.java.net) - (CDDL 1.0) JSR311 API (javax.ws.rs:jsr311-api:jar:1.1.1 - https://jsr311.dev.java.net) - -***************** -Public Domain -***************** - - The following binary components are provided to the 'Public Domain'. See project link for details. - - (Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/) - - -************************ -Eclipse Distribution License 1.0 -************************ - - The following binary components are provided under the Eclipse Distribution License 1.0. - - (EDL 1.0) Jakarta Activation API (jakarta.activation:jakarta.activation-api:jar:1.2.1) - (EDL 1.0) Jakarta XML Binding API (jakarta.xml.bind:jakarta.xml.bind-api:jar:2.3.3) - -************************ -Eclipse Public License 2.0 -************************ - - The following binary components are provided under the Eclipse Public License 2.0. - - (EPL 2.0) javax.ws.rs-api (https://github.com/eclipse-ee4j/jaxrs-api) javax.ws.rs:javax.ws.rs-api:bundle:2.1.1 - -************************ -BSD License -************************ - - (BSD) JSch - The following NOTICE information applies: - Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. - All rights reserved. - https://www.jcraft.com/jsch/ - - (BSD 3-Clause) JLine Bundle - The following NOTICE information applies: - Copyright (c) 2002-2007, Marc Prud'hommeaux. All rights reserved. - https://github.com/jline/jline1 - - (BSD 3-Clause) ThreeTen-Extra - The following NOTICE information applies: - Copyright (c) 2007-2022, Stephen Colebourne & Michael Nascimento Santos. - https://github.com/ThreeTen/threeten-extra/ - -************************ -Go License -************************ - -The following binary components are provided under the Go License. See project link for details. - - (Go) RE2/J - The following NOTICE information applies: - Copyright (c) 2009 The Go Authors. All rights reserved. - https://github.com/google/re2j \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/pom.xml deleted file mode 100644 index e16dd764b8..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/pom.xml +++ /dev/null @@ -1,335 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-processors - jar - - - - - - - org.apache.nifi - nifi-record - - - org.apache.nifi - nifi-utils - - - org.apache.nifi - nifi-record-serialization-service-api - - - org.apache.nifi - nifi-iceberg-services-api - 2.0.0-SNAPSHOT - provided - - - org.apache.nifi - nifi-iceberg-common - 2.0.0-SNAPSHOT - - - - - org.apache.hive - hive-exec - ${hive.version} - core - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - org.apache.logging.log4j - log4j-core - - - org.apache.logging.log4j - log4j-web - - - org.apache.logging.log4j - log4j-1.2-api - - - org.apache.logging.log4j - log4j-slf4j-impl - - - org.apache.hive - hive-llap-tez - - - org.apache.calcite - calcite-core - - - org.apache.calcite - calcite-druid - - - org.apache.calcite.avatica - avatica - - - com.google.guava - guava - - - org.apache.hadoop - hadoop-auth - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-server-common - - - org.apache.hadoop - hadoop-yarn-registry - - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - - - org.bouncycastle - bcprov-jdk15on - - - org.codehaus.groovy - groovy-all - - - org.apache.ivy - ivy - - - org.eclipse.jetty - jetty-server - - - - - org.bouncycastle - bcprov-jdk18on - - - org.apache.hive - hive-metastore - ${hive.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - org.apache.logging.log4j - log4j-core - - - org.apache.logging.log4j - log4j-web - - - org.apache.logging.log4j - log4j-1.2-api - - - org.apache.logging.log4j - log4j-slf4j-impl - - - org.apache.orc - orc-core - - - org.apache.hbase - hbase-client - - - org.apache.hbase - hbase-mapreduce - - - org.apache.hbase - hbase-hadoop2-compat - - - co.cask.tephra - tephra-api - - - co.cask.tephra - tephra-core - - - co.cask.tephra - tephra-hbase-compat-1.0 - - - org.apache.parquet - parquet-hadoop-bundle - - - com.tdunning - json - - - com.zaxxer - HikariCP - - - com.google.guava - guava - - - org.eclipse.jetty - jetty-xml - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-webapp - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-http - - - org.eclipse.jetty - jetty-rewrite - - - - - - org.apache.derby - derbytools - ${derby.version} - - - org.apache.derby - derby - ${derby.version} - - - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - test - - - log4j - log4j - - - commons-logging - commons-logging - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-util-ajax - - - - - org.apache.nifi - nifi-iceberg-test-utils - 2.0.0-SNAPSHOT - test - - - org.apache.nifi - nifi-avro-record-utils - 2.0.0-SNAPSHOT - test - - - org.apache.nifi - nifi-mock-record-utils - - - org.apache.nifi - nifi-kerberos-user-service-api - test - - - org.apache.nifi - nifi-iceberg-services - 2.0.0-SNAPSHOT - test - - - org.apache.nifi - nifi-dbcp-service - 2.0.0-SNAPSHOT - test - - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/PutIceberg.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/PutIceberg.java deleted file mode 100644 index cecdbff791..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/PutIceberg.java +++ /dev/null @@ -1,415 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PendingUpdate; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.CommitFailedException; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.util.Tasks; -import org.apache.nifi.annotation.behavior.DynamicProperty; -import org.apache.nifi.annotation.behavior.WritesAttribute; -import org.apache.nifi.annotation.behavior.WritesAttributes; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.ValidationContext; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.context.PropertyContext; -import org.apache.nifi.expression.ExpressionLanguageScope; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.flowfile.attributes.CoreAttributes; -import org.apache.nifi.hadoop.SecurityUtil; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.processors.iceberg.catalog.IcebergCatalogFactory; -import org.apache.nifi.processors.iceberg.converter.IcebergRecordConverter; -import org.apache.nifi.processors.iceberg.writer.IcebergTaskWriterFactory; -import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RecordReaderFactory; -import org.apache.nifi.serialization.record.Record; -import org.apache.nifi.services.iceberg.IcebergCatalogService; - -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; -import static org.apache.nifi.processors.iceberg.IcebergUtils.getConfigurationFromFiles; -import static org.apache.nifi.processors.iceberg.IcebergUtils.getDynamicProperties; - -@Tags({"iceberg", "put", "table", "store", "record", "parse", "orc", "parquet", "avro"}) -@CapabilityDescription("This processor uses Iceberg API to parse and load records into Iceberg tables. " + - "The incoming data sets are parsed with Record Reader Controller Service and ingested into an Iceberg table using the configured catalog service and provided table information. " + - "The target Iceberg table should already exist and it must have matching schemas with the incoming records, " + - "which means the Record Reader schema must contain all the Iceberg schema fields, every additional field which is not present in the Iceberg schema will be ignored. " + - "To avoid 'small file problem' it is recommended pre-appending a MergeRecord processor.") -@DynamicProperty( - name = "A custom key to add to the snapshot summary. The value must start with 'snapshot-property.' prefix.", - value = "A custom value to add to the snapshot summary.", - expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES, - description = "Adds an entry with custom-key and corresponding value in the snapshot summary. The key format must be 'snapshot-property.custom-key'.") -@WritesAttributes({ - @WritesAttribute(attribute = "iceberg.record.count", description = "The number of records in the FlowFile.") -}) -public class PutIceberg extends AbstractIcebergProcessor { - - public static final String ICEBERG_RECORD_COUNT = "iceberg.record.count"; - public static final String ICEBERG_SNAPSHOT_SUMMARY_PREFIX = "snapshot-property."; - public static final String ICEBERG_SNAPSHOT_SUMMARY_FLOWFILE_UUID = "nifi-flowfile-uuid"; - - static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder() - .name("record-reader") - .displayName("Record Reader") - .description("Specifies the Controller Service to use for parsing incoming data and determining the data's schema.") - .identifiesControllerService(RecordReaderFactory.class) - .required(true) - .build(); - - static final PropertyDescriptor CATALOG_NAMESPACE = new PropertyDescriptor.Builder() - .name("catalog-namespace") - .displayName("Catalog Namespace") - .description("The namespace of the catalog.") - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder() - .name("table-name") - .displayName("Table Name") - .description("The name of the Iceberg table to write to.") - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - static final PropertyDescriptor UNMATCHED_COLUMN_BEHAVIOR = new PropertyDescriptor.Builder() - .name("unmatched-column-behavior") - .displayName("Unmatched Column Behavior") - .description("If an incoming record does not have a field mapping for all of the database table's columns, this property specifies how to handle the situation.") - .allowableValues(UnmatchedColumnBehavior.class) - .defaultValue(UnmatchedColumnBehavior.FAIL_UNMATCHED_COLUMN) - .required(true) - .build(); - - static final PropertyDescriptor FILE_FORMAT = new PropertyDescriptor.Builder() - .name("file-format") - .displayName("File Format") - .description("File format to use when writing Iceberg data files." + - " If not set, then the 'write.format.default' table property will be used, default value is parquet.") - .allowableValues(new FileFormat[]{FileFormat.AVRO, FileFormat.PARQUET, FileFormat.ORC}) - .build(); - - static final PropertyDescriptor MAXIMUM_FILE_SIZE = new PropertyDescriptor.Builder() - .name("maximum-file-size") - .displayName("Maximum File Size") - .description("The maximum size that a file can be, if the file size is exceeded a new file will be generated with the remaining data." + - " If not set, then the 'write.target-file-size-bytes' table property will be used, default value is 512 MB.") - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .addValidator(StandardValidators.LONG_VALIDATOR) - .build(); - - static final PropertyDescriptor NUMBER_OF_COMMIT_RETRIES = new PropertyDescriptor.Builder() - .name("number-of-commit-retries") - .displayName("Number of Commit Retries") - .description("Number of times to retry a commit before failing.") - .required(true) - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .defaultValue("10") - .addValidator(StandardValidators.INTEGER_VALIDATOR) - .build(); - - static final PropertyDescriptor MINIMUM_COMMIT_WAIT_TIME = new PropertyDescriptor.Builder() - .name("minimum-commit-wait-time") - .displayName("Minimum Commit Wait Time") - .description("Minimum time to wait before retrying a commit.") - .required(true) - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .defaultValue("100 ms") - .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) - .build(); - - static final PropertyDescriptor MAXIMUM_COMMIT_WAIT_TIME = new PropertyDescriptor.Builder() - .name("maximum-commit-wait-time") - .displayName("Maximum Commit Wait Time") - .description("Maximum time to wait before retrying a commit.") - .required(true) - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .defaultValue("2 sec") - .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) - .build(); - - static final PropertyDescriptor MAXIMUM_COMMIT_DURATION = new PropertyDescriptor.Builder() - .name("maximum-commit-duration") - .displayName("Maximum Commit Duration") - .description("Total retry timeout period for a commit.") - .required(true) - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .defaultValue("30 sec") - .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) - .build(); - - static final Relationship REL_SUCCESS = new Relationship.Builder() - .name("success") - .description("A FlowFile is routed to this relationship after the data ingestion was successful.") - .build(); - - private static final List PROPERTIES = List.of( - RECORD_READER, - CATALOG, - CATALOG_NAMESPACE, - TABLE_NAME, - UNMATCHED_COLUMN_BEHAVIOR, - FILE_FORMAT, - MAXIMUM_FILE_SIZE, - KERBEROS_USER_SERVICE, - NUMBER_OF_COMMIT_RETRIES, - MINIMUM_COMMIT_WAIT_TIME, - MAXIMUM_COMMIT_WAIT_TIME, - MAXIMUM_COMMIT_DURATION - ); - - public static final Set RELATIONSHIPS = Set.of(REL_SUCCESS, REL_FAILURE); - - @Override - protected List getSupportedPropertyDescriptors() { - return PROPERTIES; - } - - @Override - public Set getRelationships() { - return RELATIONSHIPS; - } - - @Override - protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { - return new PropertyDescriptor.Builder() - .name(propertyDescriptorName) - .required(false) - .addValidator((subject, input, context) -> { - ValidationResult.Builder builder = new ValidationResult.Builder().subject(subject).input(input); - if (subject.startsWith(ICEBERG_SNAPSHOT_SUMMARY_PREFIX)) { - builder.valid(true); - } else { - builder.valid(false).explanation("Dynamic property key must begin with '" + ICEBERG_SNAPSHOT_SUMMARY_PREFIX + "'"); - } - return builder.build(); - }) - .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) - .dynamic(true) - .build(); - } - - @Override - protected Collection customValidate(ValidationContext context) { - final List problems = new ArrayList<>(); - final IcebergCatalogService catalogService = context.getProperty(CATALOG).asControllerService(IcebergCatalogService.class); - boolean catalogServiceEnabled = context.getControllerServiceLookup().isControllerServiceEnabled(catalogService); - - if (catalogServiceEnabled) { - final boolean kerberosUserServiceIsSet = context.getProperty(KERBEROS_USER_SERVICE).isSet(); - final boolean securityEnabled = SecurityUtil.isSecurityEnabled(getConfigurationFromFiles(catalogService.getConfigFilePaths())); - - if (securityEnabled && !kerberosUserServiceIsSet) { - problems.add(new ValidationResult.Builder() - .subject(KERBEROS_USER_SERVICE.getDisplayName()) - .valid(false) - .explanation("'hadoop.security.authentication' is set to 'kerberos' in the hadoop configuration files but no KerberosUserService is configured.") - .build()); - } - - if (!securityEnabled && kerberosUserServiceIsSet) { - problems.add(new ValidationResult.Builder() - .subject(KERBEROS_USER_SERVICE.getDisplayName()) - .valid(false) - .explanation("KerberosUserService is configured but 'hadoop.security.authentication' is not set to 'kerberos' in the hadoop configuration files.") - .build()); - } - } - - return problems; - } - - @Override - public void doOnTrigger(ProcessContext context, ProcessSession session, FlowFile flowFile) throws ProcessException { - final long startNanos = System.nanoTime(); - final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); - final String fileFormat = context.getProperty(FILE_FORMAT).getValue(); - final String maximumFileSize = context.getProperty(MAXIMUM_FILE_SIZE).evaluateAttributeExpressions(flowFile).getValue(); - - Catalog catalog = null; - Table table = null; - TaskWriter taskWriter = null; - int recordCount = 0; - - try (final InputStream in = session.read(flowFile); - final RecordReader reader = readerFactory.createRecordReader(flowFile, in, getLogger())) { - catalog = loadCatalog(context); - table = loadTable(context, flowFile, catalog); - final FileFormat format = getFileFormat(table.properties(), fileFormat); - final IcebergTaskWriterFactory taskWriterFactory = new IcebergTaskWriterFactory(table, flowFile.getId(), format, maximumFileSize); - taskWriter = taskWriterFactory.create(); - final UnmatchedColumnBehavior unmatchedColumnBehavior = context.getProperty(UNMATCHED_COLUMN_BEHAVIOR).asAllowableValue(UnmatchedColumnBehavior.class); - - final IcebergRecordConverter recordConverter = new IcebergRecordConverter(table.schema(), reader.getSchema(), format, unmatchedColumnBehavior, getLogger()); - - Record record; - while ((record = reader.nextRecord()) != null) { - taskWriter.write(recordConverter.convert(record)); - recordCount++; - } - - final WriteResult result = taskWriter.complete(); - appendDataFiles(context, flowFile, table, result); - } catch (Exception e) { - if (!handleAuthErrors(e, session, context)) { - getLogger().error("Exception occurred while writing Iceberg records", e); - session.transfer(session.penalize(flowFile), REL_FAILURE); - } - - try { - if (taskWriter != null) { - abort(taskWriter.dataFiles(), table); - } - } catch (Exception ex) { - getLogger().warn("Failed to abort uncommitted data files", ex); - } - return; - } finally { - if (catalog instanceof Closeable) { - try { - ((Closeable) catalog).close(); - } catch (IOException e) { - getLogger().warn("Failed to close catalog", e); - } - } - } - - flowFile = session.putAttribute(flowFile, ICEBERG_RECORD_COUNT, String.valueOf(recordCount)); - final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); - session.getProvenanceReporter().send(flowFile, table.location(), transferMillis); - session.transfer(flowFile, REL_SUCCESS); - } - - /** - * Loads a table from the catalog service with the provided values from the property context. - * - * @param context holds the user provided information for the {@link Catalog} and the {@link Table} - * @return loaded table - */ - private Table loadTable(final PropertyContext context, final FlowFile flowFile, final Catalog catalog) { - final String catalogNamespace = context.getProperty(CATALOG_NAMESPACE).evaluateAttributeExpressions(flowFile).getValue(); - final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue(); - - final Namespace namespace = Namespace.of(catalogNamespace); - final TableIdentifier tableIdentifier = TableIdentifier.of(namespace, tableName); - - return catalog.loadTable(tableIdentifier); - } - - private Catalog loadCatalog(final PropertyContext context) { - final IcebergCatalogService catalogService = context.getProperty(CATALOG).asControllerService(IcebergCatalogService.class); - final IcebergCatalogFactory catalogFactory = new IcebergCatalogFactory(catalogService); - - return catalogFactory.create(); - } - - /** - * Appends the pending data files to the given {@link Table}. - * - * @param context processor context - * @param table table to append - * @param result datafiles created by the {@link TaskWriter} - */ - void appendDataFiles(ProcessContext context, FlowFile flowFile, Table table, WriteResult result) { - final int numberOfCommitRetries = context.getProperty(NUMBER_OF_COMMIT_RETRIES).evaluateAttributeExpressions(flowFile).asInteger(); - final long minimumCommitWaitTime = context.getProperty(MINIMUM_COMMIT_WAIT_TIME).evaluateAttributeExpressions(flowFile).asTimePeriod(TimeUnit.MILLISECONDS); - final long maximumCommitWaitTime = context.getProperty(MAXIMUM_COMMIT_WAIT_TIME).evaluateAttributeExpressions(flowFile).asTimePeriod(TimeUnit.MILLISECONDS); - final long maximumCommitDuration = context.getProperty(MAXIMUM_COMMIT_DURATION).evaluateAttributeExpressions(flowFile).asTimePeriod(TimeUnit.MILLISECONDS); - - final AppendFiles appender = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appender::appendFile); - - addSnapshotSummaryProperties(context, appender, flowFile); - - Tasks.foreach(appender) - .exponentialBackoff(minimumCommitWaitTime, maximumCommitWaitTime, maximumCommitDuration, 2.0) - .retry(numberOfCommitRetries) - .onlyRetryOn(CommitFailedException.class) - .run(PendingUpdate::commit); - } - - /** - * Adds the FlowFile's uuid and additional entries provided in Dynamic properties to the snapshot summary. - * - * @param context processor context - * @param appender table appender to set the snapshot summaries - * @param flowFile the FlowFile to get the uuid from - */ - private void addSnapshotSummaryProperties(ProcessContext context, AppendFiles appender, FlowFile flowFile) { - appender.set(ICEBERG_SNAPSHOT_SUMMARY_FLOWFILE_UUID, flowFile.getAttribute(CoreAttributes.UUID.key())); - - for (Map.Entry dynamicProperty : getDynamicProperties(context, flowFile).entrySet()) { - String key = dynamicProperty.getKey().substring(ICEBERG_SNAPSHOT_SUMMARY_PREFIX.length()); - appender.set(key, dynamicProperty.getValue()); - } - } - - /** - * Determines the write file format from the requested value and the table configuration. - * - * @param tableProperties table properties - * @param fileFormat requested file format from the processor - * @return file format to use - */ - private FileFormat getFileFormat(Map tableProperties, String fileFormat) { - final String fileFormatName = fileFormat != null ? fileFormat : tableProperties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); - return FileFormat.valueOf(fileFormatName.toUpperCase(Locale.ENGLISH)); - } - - /** - * Deletes the completed data files that have not been committed to the table yet. - * - * @param dataFiles files created by the task writer - * @param table table - */ - void abort(DataFile[] dataFiles, Table table) { - Tasks.foreach(dataFiles) - .retry(3) - .run(file -> table.io().deleteFile(file.path().toString())); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergPartitionedWriter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergPartitionedWriter.java deleted file mode 100644 index 0b3f98e7c5..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergPartitionedWriter.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.writer; - -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.PartitionedFanoutWriter; - -/** - * This class adapts {@link Record} for partitioned writing. - */ -public class IcebergPartitionedWriter extends PartitionedFanoutWriter { - - private final PartitionKey partitionKey; - private final InternalRecordWrapper wrapper; - - public IcebergPartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, - FileIO io, long targetFileSize, Schema schema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.partitionKey = new PartitionKey(spec, schema); - this.wrapper = new InternalRecordWrapper(schema.asStruct()); - } - - @Override - protected PartitionKey partition(Record record) { - partitionKey.partition(wrapper.wrap(record)); - return partitionKey; - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergTaskWriterFactory.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergTaskWriterFactory.java deleted file mode 100644 index 4dc2bbb6c0..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/writer/IcebergTaskWriterFactory.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.writer; - -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.UnpartitionedWriter; -import org.apache.iceberg.util.PropertyUtil; - -/** - * Factory class to create the suitable {@link TaskWriter} based on the {@link Table}'s properties - */ -public class IcebergTaskWriterFactory { - - private final Schema schema; - private final PartitionSpec spec; - private final FileIO io; - private final long targetFileSize; - private final FileFormat fileFormat; - private final FileAppenderFactory appenderFactory; - private final OutputFileFactory outputFileFactory; - - public IcebergTaskWriterFactory(Table table, long taskId, FileFormat fileFormat, String targetFileSize) { - this.schema = table.schema(); - this.spec = table.spec(); - this.io = table.io(); - this.fileFormat = fileFormat; - - this.targetFileSize = targetFileSize != null ? Long.parseLong(targetFileSize) : - PropertyUtil.propertyAsLong(table.properties(), TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); - - this.outputFileFactory = OutputFileFactory.builderFor(table, table.spec().specId(), taskId).format(fileFormat).build(); - this.appenderFactory = new GenericAppenderFactory(schema, spec); - } - - public TaskWriter create() { - if (spec.isUnpartitioned()) { - return new UnpartitionedWriter<>(spec, fileFormat, appenderFactory, outputFileFactory, io, targetFileSize); - } else { - return new IcebergPartitionedWriter(spec, fileFormat, appenderFactory, outputFileFactory, io, targetFileSize, schema); - } - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor deleted file mode 100644 index f884831b07..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ /dev/null @@ -1,16 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.nifi.processors.iceberg.PutIceberg diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/docs/org.apache.nifi.processors.iceberg.PutIceberg/additionalDetails.md b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/docs/org.apache.nifi.processors.iceberg.PutIceberg/additionalDetails.md deleted file mode 100644 index 3870e91e5d..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/resources/docs/org.apache.nifi.processors.iceberg.PutIceberg/additionalDetails.md +++ /dev/null @@ -1,46 +0,0 @@ - - -# PutIceberg - -### Description - -Iceberg is a high-performance format for huge analytic tables. The PutIceberg processor is capable of pushing data into -Iceberg tables using different types of Iceberg catalog implementations. - -### Commit retry properties - -Iceberg supports multiple concurrent writes using optimistic concurrency. The processor's commit retry implementation is -using **exponential backoff** with **jitter** and **scale factor 2**, and provides the following properties to configure -the behaviour according to its usage. - -* Number Of Commit Retries (default: 10) - Number of retries that the processor is going to try to commit the new data - files. -* Minimum Commit Wait Time (default: 100 ms) - Minimum time that the processor is going to wait before each commit - attempt. -* Maximum Commit Wait Time (default: 2 sec) - Maximum time that the processor is going to wait before each commit - attempt. -* Maximum Commit Duration (default: 30 sec) - Maximum duration that the processor is going to wait before failing the - current processor event's commit. - -The NiFi side retry logic is built on top of the Iceberg commit retry logic which can be configured through table -properties. See -more: [Table behavior properties](https://iceberg.apache.org/docs/latest/configuration/#table-behavior-properties) - -### Snapshot summary properties - -The processor provides an option to add additional properties to the snapshot summary using dynamic properties. The -additional property must have the 'snapshot-property.' prefix in the dynamic property key but the actual entry will be -inserted without it. Each snapshot automatically gets the FlowFile's uuid in the 'nifi-flowfile-uuid' summary property. \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/AbstractTestPutIceberg.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/AbstractTestPutIceberg.java deleted file mode 100644 index cd1c5f37c8..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/AbstractTestPutIceberg.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.avro.Schema; -import org.apache.commons.io.IOUtils; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; -import org.apache.nifi.avro.AvroTypeUtil; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.serialization.record.MockRecordParser; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.util.TestRunner; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Paths; - -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.NAMESPACE; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.RECORD_READER_SERVICE; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.createTemporaryDirectory; - -public class AbstractTestPutIceberg { - - protected static final String TABLE_NAME = "users"; - - protected static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(NAMESPACE, TABLE_NAME); - - protected static final org.apache.iceberg.Schema USER_SCHEMA = new org.apache.iceberg.Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "name", Types.StringType.get()), - Types.NestedField.required(3, "department", Types.StringType.get()) - ); - - protected TestRunner runner; - protected PutIceberg processor; - protected Catalog catalog; - protected String warehousePath; - protected static Schema inputSchema; - - protected void initRecordReader() throws InitializationException { - final MockRecordParser readerFactory = new MockRecordParser(); - final RecordSchema recordSchema = AvroTypeUtil.createSchema(inputSchema); - - for (RecordField recordField : recordSchema.getFields()) { - readerFactory.addSchemaField(recordField); - } - - readerFactory.addRecord(0, "John", "Finance"); - readerFactory.addRecord(1, "Jill", "Finance"); - readerFactory.addRecord(2, "James", "Marketing"); - readerFactory.addRecord(3, "Joana", "Sales"); - - runner.addControllerService(RECORD_READER_SERVICE, readerFactory); - runner.enableControllerService(readerFactory); - - runner.setProperty(PutIceberg.RECORD_READER, RECORD_READER_SERVICE); - } - - @BeforeAll - public static void initSchema() throws IOException { - final String avroSchema = IOUtils.toString(Files.newInputStream(Paths.get("src/test/resources/user.avsc")), StandardCharsets.UTF_8); - inputSchema = new Schema.Parser().parse(avroSchema); - } - - @BeforeEach - public void setUp() { - warehousePath = createTemporaryDirectory().getAbsolutePath(); - processor = new PutIceberg(); - } - - @AfterEach - public void tearDown() { - catalog.dropTable(TABLE_IDENTIFIER); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestDataFileActions.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestDataFileActions.java deleted file mode 100644 index 6bdd7dd5f7..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestDataFileActions.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.exceptions.CommitFailedException; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.types.Types; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processors.iceberg.converter.IcebergRecordConverter; -import org.apache.nifi.processors.iceberg.writer.IcebergTaskWriterFactory; -import org.apache.nifi.serialization.SimpleRecordSchema; -import org.apache.nifi.serialization.record.MapRecord; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordFieldType; -import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.util.MockComponentLog; -import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.MockPropertyValue; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.DisabledOnOs; -import org.mockito.Mockito; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Random; - -import static org.apache.nifi.processors.iceberg.PutIceberg.MAXIMUM_COMMIT_DURATION; -import static org.apache.nifi.processors.iceberg.PutIceberg.MAXIMUM_COMMIT_WAIT_TIME; -import static org.apache.nifi.processors.iceberg.PutIceberg.MINIMUM_COMMIT_WAIT_TIME; -import static org.apache.nifi.processors.iceberg.PutIceberg.NUMBER_OF_COMMIT_RETRIES; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.createTemporaryDirectory; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.condition.OS.WINDOWS; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class TestDataFileActions { - - private static final Namespace NAMESPACE = Namespace.of("default"); - private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(NAMESPACE, "abort"); - - private static final Schema ABORT_SCHEMA = new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get()) - ); - - private PutIceberg icebergProcessor; - private ComponentLog logger; - - @BeforeEach - public void setUp() { - icebergProcessor = new PutIceberg(); - logger = new MockComponentLog("id", "TestDataFileActions"); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testAbortUncommittedFiles() throws IOException { - Table table = initCatalog(); - - List recordFields = Collections.singletonList(new RecordField("id", RecordFieldType.INT.getDataType())); - RecordSchema abortSchema = new SimpleRecordSchema(recordFields); - - List recordList = new ArrayList<>(); - recordList.add(new MapRecord(abortSchema, Collections.singletonMap("id", 1))); - recordList.add(new MapRecord(abortSchema, Collections.singletonMap("id", 2))); - recordList.add(new MapRecord(abortSchema, Collections.singletonMap("id", 3))); - recordList.add(new MapRecord(abortSchema, Collections.singletonMap("id", 4))); - recordList.add(new MapRecord(abortSchema, Collections.singletonMap("id", 5))); - - IcebergTaskWriterFactory taskWriterFactory = new IcebergTaskWriterFactory(table, new Random().nextLong(), FileFormat.PARQUET, null); - TaskWriter taskWriter = taskWriterFactory.create(); - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(table.schema(), abortSchema, FileFormat.PARQUET, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - - for (MapRecord record : recordList) { - taskWriter.write(recordConverter.convert(record)); - } - - DataFile[] dataFiles = taskWriter.dataFiles(); - - // DataFiles written by the taskWriter should exist - for (DataFile dataFile : dataFiles) { - Assertions.assertTrue(Files.exists(Paths.get(dataFile.path().toString()))); - } - - icebergProcessor.abort(taskWriter.dataFiles(), table); - - // DataFiles shouldn't exist after aborting them - for (DataFile dataFile : dataFiles) { - Assertions.assertFalse(Files.exists(Paths.get(dataFile.path().toString()))); - } - } - - @Test - public void testAppenderCommitRetryExceeded() { - ProcessContext context = Mockito.mock(ProcessContext.class); - when(context.getProperty(NUMBER_OF_COMMIT_RETRIES)).thenReturn(new MockPropertyValue("3", null)); - when(context.getProperty(MINIMUM_COMMIT_WAIT_TIME)).thenReturn(new MockPropertyValue("1 ms", null)); - when(context.getProperty(MAXIMUM_COMMIT_WAIT_TIME)).thenReturn(new MockPropertyValue("1 ms", null)); - when(context.getProperty(MAXIMUM_COMMIT_DURATION)).thenReturn(new MockPropertyValue("1 min", null)); - - FlowFile mockFlowFile = new MockFlowFile(1234567890L); - AppendFiles appender = Mockito.mock(AppendFiles.class); - doThrow(CommitFailedException.class).when(appender).commit(); - - Table table = Mockito.mock(Table.class); - when(table.newAppend()).thenReturn(appender); - - // assert the commit action eventually fails after exceeding the number of retries - assertThrows(CommitFailedException.class, () -> icebergProcessor.appendDataFiles(context, mockFlowFile, table, WriteResult.builder().build())); - - // verify the commit action was called the configured number of times - verify(appender, times(4)).commit(); - } - - @SuppressWarnings("unchecked") - @Test - public void testAppenderCommitSucceeded() { - ProcessContext context = Mockito.mock(ProcessContext.class); - when(context.getProperty(NUMBER_OF_COMMIT_RETRIES)).thenReturn(new MockPropertyValue("3", null)); - when(context.getProperty(MINIMUM_COMMIT_WAIT_TIME)).thenReturn(new MockPropertyValue("1 ms", null)); - when(context.getProperty(MAXIMUM_COMMIT_WAIT_TIME)).thenReturn(new MockPropertyValue("1 ms", null)); - when(context.getProperty(MAXIMUM_COMMIT_DURATION)).thenReturn(new MockPropertyValue("1 min", null)); - - FlowFile mockFlowFile = new MockFlowFile(1234567890L); - AppendFiles appender = Mockito.mock(AppendFiles.class); - // the commit action should throw exception 2 times before succeeding - doThrow(CommitFailedException.class, CommitFailedException.class).doNothing().when(appender).commit(); - - Table table = Mockito.mock(Table.class); - when(table.newAppend()).thenReturn(appender); - - // the method call shouldn't throw exception since the configured number of retries is higher than the number of failed commit actions - icebergProcessor.appendDataFiles(context, mockFlowFile, table, WriteResult.builder().build()); - - // verify the proper number of commit action was called - verify(appender, times(3)).commit(); - } - - @Test - public void testMaxCommitDurationExceeded() { - ProcessContext context = Mockito.mock(ProcessContext.class); - when(context.getProperty(NUMBER_OF_COMMIT_RETRIES)).thenReturn(new MockPropertyValue("5", null)); - when(context.getProperty(MINIMUM_COMMIT_WAIT_TIME)).thenReturn(new MockPropertyValue("2 ms", null)); - when(context.getProperty(MAXIMUM_COMMIT_WAIT_TIME)).thenReturn(new MockPropertyValue("2 ms", null)); - when(context.getProperty(MAXIMUM_COMMIT_DURATION)).thenReturn(new MockPropertyValue("1 ms", null)); - - FlowFile mockFlowFile = new MockFlowFile(1234567890L); - AppendFiles appender = Mockito.mock(AppendFiles.class); - doThrow(CommitFailedException.class).when(appender).commit(); - - Table table = Mockito.mock(Table.class); - when(table.newAppend()).thenReturn(appender); - - // assert the commit action eventually fails after exceeding duration of maximum retries - assertThrows(CommitFailedException.class, () -> icebergProcessor.appendDataFiles(context, mockFlowFile, table, WriteResult.builder().build())); - - // verify the commit action was called only 2 times instead of the configured 5 - verify(appender, times(2)).commit(); - } - - private Table initCatalog() { - final File warehousePath = createTemporaryDirectory(); - final HadoopCatalog catalog = new HadoopCatalog(new Configuration(), warehousePath.getAbsolutePath()); - - return catalog.createTable(TABLE_IDENTIFIER, ABORT_SCHEMA, PartitionSpec.unpartitioned()); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestIcebergRecordConverter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestIcebergRecordConverter.java deleted file mode 100644 index 4db691b072..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestIcebergRecordConverter.java +++ /dev/null @@ -1,1124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.avro.AvroIterable; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.avro.DataReader; -import org.apache.iceberg.data.avro.DataWriter; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.orc.GenericOrcWriter; -import org.apache.iceberg.data.parquet.GenericParquetReaders; -import org.apache.iceberg.data.parquet.GenericParquetWriter; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processors.iceberg.converter.ArrayElementGetter; -import org.apache.nifi.processors.iceberg.converter.IcebergRecordConverter; -import org.apache.nifi.processors.iceberg.converter.RecordFieldGetter; -import org.apache.nifi.serialization.SimpleRecordSchema; -import org.apache.nifi.serialization.record.DataType; -import org.apache.nifi.serialization.record.MapRecord; -import org.apache.nifi.serialization.record.Record; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordFieldType; -import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.serialization.record.type.ArrayDataType; -import org.apache.nifi.serialization.record.type.MapDataType; -import org.apache.nifi.serialization.record.type.RecordDataType; -import org.apache.nifi.util.MockComponentLog; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.DisabledOnOs; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -import static java.io.File.createTempFile; -import static org.apache.iceberg.FileFormat.PARQUET; -import static org.junit.jupiter.api.Assertions.assertArrayEquals; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertInstanceOf; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.condition.OS.WINDOWS; - -public class TestIcebergRecordConverter { - - private static final LocalDateTime LOCAL_DATE_TIME = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000); - - private OutputFile tempFile; - - private ComponentLog logger; - - @BeforeEach - public void setUp() throws Exception { - tempFile = Files.localOutput(createTempFile("test", null)); - logger = new MockComponentLog("id", "TestIcebergRecordConverter"); - } - - @AfterEach - public void tearDown() { - File file = new File(tempFile.location()); - file.deleteOnExit(); - } - - private static final Schema STRUCT_SCHEMA = new Schema( - Types.NestedField.required(0, "struct", Types.StructType.of( - Types.NestedField.required(1, "nested_struct", Types.StructType.of( - Types.NestedField.required(2, "string", Types.StringType.get()), - Types.NestedField.required(3, "integer", Types.IntegerType.get())) - ) - )) - ); - - private static final Schema LIST_SCHEMA = new Schema( - Types.NestedField.required(0, "list", Types.ListType.ofRequired( - 1, Types.ListType.ofRequired(2, Types.StringType.get()) - ) - ) - ); - - private static final Schema MAP_SCHEMA = new Schema( - Types.NestedField.required(0, "map", Types.MapType.ofRequired( - 1, 2, Types.StringType.get(), Types.MapType.ofRequired( - 3, 4, Types.StringType.get(), Types.LongType.get() - ) - )) - ); - - private static final Schema RECORD_IN_LIST_SCHEMA = new Schema( - Types.NestedField.required(0, "list", Types.ListType.ofRequired( - 1, Types.StructType.of( - Types.NestedField.required(2, "string", Types.StringType.get()), - Types.NestedField.required(3, "integer", Types.IntegerType.get()))) - ) - ); - - private static final Schema RECORD_IN_MAP_SCHEMA = new Schema( - Types.NestedField.required(0, "map", Types.MapType.ofRequired( - 1, 2, Types.StringType.get(), Types.StructType.of( - Types.NestedField.required(3, "string", Types.StringType.get()), - Types.NestedField.required(4, "integer", Types.IntegerType.get()))) - ) - ); - - private static final Schema PRIMITIVES_SCHEMA = new Schema( - Types.NestedField.optional(0, "string", Types.StringType.get()), - Types.NestedField.optional(1, "integer", Types.IntegerType.get()), - Types.NestedField.optional(2, "float", Types.FloatType.get()), - Types.NestedField.optional(3, "long", Types.LongType.get()), - Types.NestedField.optional(4, "double", Types.DoubleType.get()), - Types.NestedField.optional(5, "decimal", Types.DecimalType.of(10, 2)), - Types.NestedField.optional(6, "decimalLowerScore", Types.DecimalType.of(10, 2)), - Types.NestedField.optional(7, "boolean", Types.BooleanType.get()), - Types.NestedField.optional(8, "fixed", Types.FixedType.ofLength(5)), - Types.NestedField.optional(9, "binary", Types.BinaryType.get()), - Types.NestedField.optional(10, "date", Types.DateType.get()), - Types.NestedField.optional(11, "time", Types.TimeType.get()), - Types.NestedField.optional(12, "timestamp", Types.TimestampType.withZone()), - Types.NestedField.optional(13, "timestampTz", Types.TimestampType.withoutZone()), - Types.NestedField.optional(14, "uuid", Types.UUIDType.get()), - Types.NestedField.optional(15, "choice", Types.IntegerType.get()), - Types.NestedField.optional(16, "enum", Types.StringType.get()) - ); - - private static final Schema PRIMITIVES_SCHEMA_WITH_REQUIRED_FIELDS = new Schema( - Types.NestedField.optional(0, "string", Types.StringType.get()), - Types.NestedField.required(1, "integer", Types.IntegerType.get()), - Types.NestedField.required(2, "float", Types.FloatType.get()), - Types.NestedField.required(3, "long", Types.LongType.get()), - Types.NestedField.optional(4, "double", Types.DoubleType.get()), - Types.NestedField.optional(5, "decimal", Types.DecimalType.of(10, 2)), - Types.NestedField.optional(6, "boolean", Types.BooleanType.get()), - Types.NestedField.optional(7, "fixed", Types.FixedType.ofLength(5)), - Types.NestedField.optional(8, "binary", Types.BinaryType.get()), - Types.NestedField.optional(9, "date", Types.DateType.get()), - Types.NestedField.optional(10, "time", Types.TimeType.get()), - Types.NestedField.optional(11, "timestamp", Types.TimestampType.withZone()), - Types.NestedField.optional(12, "timestampTz", Types.TimestampType.withoutZone()), - Types.NestedField.optional(13, "uuid", Types.UUIDType.get()), - Types.NestedField.optional(14, "choice", Types.IntegerType.get()) - ); - - private static final Schema COMPATIBLE_PRIMITIVES_SCHEMA = new Schema( - Types.NestedField.optional(0, "string", Types.StringType.get()), - Types.NestedField.optional(1, "integer", Types.IntegerType.get()), - Types.NestedField.optional(2, "float", Types.FloatType.get()), - Types.NestedField.optional(3, "long", Types.LongType.get()), - Types.NestedField.optional(4, "double", Types.DoubleType.get()), - Types.NestedField.optional(5, "decimal", Types.DecimalType.of(10, 2)), - Types.NestedField.optional(7, "fixed", Types.FixedType.ofLength(5)), - Types.NestedField.optional(8, "binary", Types.BinaryType.get()), - Types.NestedField.optional(9, "date", Types.DateType.get()), - Types.NestedField.optional(10, "time", Types.TimeType.get()), - Types.NestedField.optional(11, "timestamp", Types.TimestampType.withZone()), - Types.NestedField.optional(12, "timestampTz", Types.TimestampType.withoutZone()), - Types.NestedField.optional(13, "uuid", Types.UUIDType.get()), - Types.NestedField.optional(14, "choice", Types.IntegerType.get()) - ); - - private static final Schema CASE_INSENSITIVE_SCHEMA = new Schema( - Types.NestedField.optional(0, "FIELD1", Types.StringType.get()), - Types.NestedField.optional(1, "Field2", Types.StringType.get()), - Types.NestedField.optional(2, "fielD3", Types.StringType.get()), - Types.NestedField.optional(3, "field4", Types.StringType.get()) - ); - - private static final Schema UNORDERED_SCHEMA = new Schema( - Types.NestedField.optional(0, "field1", Types.StringType.get()), - Types.NestedField.required(1, "field2", Types.StructType.of( - Types.NestedField.required(2, "field3", Types.StringType.get()), - Types.NestedField.required(3, "field4", Types.ListType.ofRequired(4, Types.StringType.get())) - )), - Types.NestedField.optional(5, "field5", Types.StringType.get()), - Types.NestedField.required(6, "field6", Types.MapType.ofRequired( - 7, 8, Types.StringType.get(), Types.StringType.get() - )) - ); - - private static RecordSchema getStructSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("struct", new RecordDataType(getNestedStructSchema()))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getNestedStructSchema() { - List nestedFields = new ArrayList<>(); - nestedFields.add(new RecordField("nested_struct", new RecordDataType(getNestedStructSchema2()))); - - return new SimpleRecordSchema(nestedFields); - } - - private static RecordSchema getNestedStructSchema2() { - List nestedFields2 = new ArrayList<>(); - nestedFields2.add(new RecordField("string", RecordFieldType.STRING.getDataType())); - nestedFields2.add(new RecordField("integer", RecordFieldType.INT.getDataType())); - - return new SimpleRecordSchema(nestedFields2); - } - - private static RecordSchema getListSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("list", new ArrayDataType( - new ArrayDataType(RecordFieldType.STRING.getDataType())))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getMapSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("map", new MapDataType( - new MapDataType(RecordFieldType.LONG.getDataType())))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getRecordInListSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("list", new ArrayDataType( - new RecordDataType(getNestedStructSchema2())))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getRecordInMapSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("map", new MapDataType( - new RecordDataType(getNestedStructSchema2())))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getPrimitivesSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("string", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("integer", RecordFieldType.INT.getDataType())); - fields.add(new RecordField("float", RecordFieldType.FLOAT.getDataType())); - fields.add(new RecordField("long", RecordFieldType.LONG.getDataType())); - fields.add(new RecordField("double", RecordFieldType.DOUBLE.getDataType())); - fields.add(new RecordField("decimal", RecordFieldType.DECIMAL.getDecimalDataType(10, 2))); - fields.add(new RecordField("decimalLowerScore", RecordFieldType.DECIMAL.getDecimalDataType(10, 2))); - fields.add(new RecordField("boolean", RecordFieldType.BOOLEAN.getDataType())); - fields.add(new RecordField("fixed", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); - fields.add(new RecordField("binary", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); - fields.add(new RecordField("date", RecordFieldType.DATE.getDataType())); - fields.add(new RecordField("time", RecordFieldType.TIME.getDataType())); - fields.add(new RecordField("timestamp", RecordFieldType.TIMESTAMP.getDataType())); - fields.add(new RecordField("timestampTz", RecordFieldType.TIMESTAMP.getDataType())); - fields.add(new RecordField("uuid", RecordFieldType.UUID.getDataType())); - fields.add(new RecordField("choice", RecordFieldType.CHOICE.getChoiceDataType(RecordFieldType.STRING.getDataType(), RecordFieldType.INT.getDataType()))); - fields.add(new RecordField("enum", RecordFieldType.ENUM.getEnumDataType(Arrays.asList("red", "blue", "yellow")))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getPrimitivesSchemaMissingFields() { - List fields = new ArrayList<>(); - fields.add(new RecordField("string", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("double", RecordFieldType.DOUBLE.getDataType())); - fields.add(new RecordField("decimal", RecordFieldType.DECIMAL.getDecimalDataType(10, 2))); - fields.add(new RecordField("boolean", RecordFieldType.BOOLEAN.getDataType())); - fields.add(new RecordField("fixed", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); - fields.add(new RecordField("binary", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); - fields.add(new RecordField("date", RecordFieldType.DATE.getDataType())); - fields.add(new RecordField("time", RecordFieldType.TIME.getDataType())); - fields.add(new RecordField("timestamp", RecordFieldType.TIMESTAMP.getDataType())); - fields.add(new RecordField("timestampTz", RecordFieldType.TIMESTAMP.getDataType())); - fields.add(new RecordField("uuid", RecordFieldType.UUID.getDataType())); - fields.add(new RecordField("choice", RecordFieldType.CHOICE.getChoiceDataType(RecordFieldType.STRING.getDataType(), RecordFieldType.INT.getDataType()))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getPrimitivesAsCompatiblesSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("string", RecordFieldType.INT.getDataType())); - fields.add(new RecordField("integer", RecordFieldType.SHORT.getDataType())); - fields.add(new RecordField("float", RecordFieldType.DOUBLE.getDataType())); - fields.add(new RecordField("long", RecordFieldType.INT.getDataType())); - fields.add(new RecordField("double", RecordFieldType.FLOAT.getDataType())); - fields.add(new RecordField("decimal", RecordFieldType.DOUBLE.getDataType())); - fields.add(new RecordField("fixed", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); - fields.add(new RecordField("binary", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); - fields.add(new RecordField("date", RecordFieldType.STRING.getDataType("yyyy-MM-dd"))); - fields.add(new RecordField("time", RecordFieldType.STRING.getDataType("HH:mm:ss.SSS"))); - fields.add(new RecordField("timestamp", RecordFieldType.STRING.getDataType("yyyy-MM-dd HH:mm:ss.SSSZ"))); - fields.add(new RecordField("timestampTz", RecordFieldType.STRING.getDataType("yyyy-MM-dd HH:mm:ss.SSSZ"))); - fields.add(new RecordField("uuid", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("choice", RecordFieldType.CHOICE.getChoiceDataType(RecordFieldType.STRING.getDataType(), RecordFieldType.INT.getDataType()))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getCaseInsensitiveSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("field1", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("FIELD2", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("Field3", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("fielD4", RecordFieldType.STRING.getDataType())); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getNestedUnorderedSchema() { - List nestedFields = new ArrayList<>(); - nestedFields.add(new RecordField("FIELD4", new ArrayDataType(RecordFieldType.STRING.getDataType()))); - nestedFields.add(new RecordField("FIELD3", RecordFieldType.STRING.getDataType())); - - return new SimpleRecordSchema(nestedFields); - } - - private static RecordSchema getUnorderedSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("FIELD6", new MapDataType(RecordFieldType.STRING.getDataType()))); - fields.add(new RecordField("FIELD5", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("FIELD1", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("FIELD2", new RecordDataType(getNestedUnorderedSchema()))); - - return new SimpleRecordSchema(fields); - } - - private static RecordSchema getChoiceSchema() { - List fields = new ArrayList<>(); - fields.add(new RecordField("string", RecordFieldType.INT.getDataType())); - fields.add(new RecordField("integer", RecordFieldType.STRING.getDataType())); - fields.add(new RecordField("float", RecordFieldType.LONG.getDataType())); - - return new SimpleRecordSchema(fields); - } - - private static Record setupStructTestRecord() { - Map nestedValues2 = new HashMap<>(); - nestedValues2.put("string", "Test String"); - nestedValues2.put("integer", 10); - MapRecord nestedRecord2 = new MapRecord(getNestedStructSchema2(), nestedValues2); - - Map nestedValues = new HashMap<>(); - nestedValues.put("nested_struct", nestedRecord2); - MapRecord nestedRecord = new MapRecord(getNestedStructSchema(), nestedValues); - - Map values = new HashMap<>(); - values.put("struct", nestedRecord); - return new MapRecord(getStructSchema(), values); - } - - private static Record setupListTestRecord() { - List nestedList1 = new ArrayList<>(); - nestedList1.add("Test String1"); - nestedList1.add("Test String2"); - - List nestedList2 = new ArrayList<>(); - nestedList2.add("Test String3"); - nestedList2.add("Test String4"); - - List> list = new ArrayList<>(); - list.add(nestedList1); - list.add(nestedList2); - - Map values = new HashMap<>(); - values.put("list", list); - - return new MapRecord(getListSchema(), values); - } - - private static Record setupMapTestRecord() { - Map nestedMap = new HashMap<>(); - nestedMap.put("nested_key", 42L); - - Map> map = new HashMap<>(); - map.put("key", nestedMap); - - Map values = new HashMap<>(); - values.put("map", map); - - return new MapRecord(getMapSchema(), values); - } - - private static Record setupRecordInListTestRecord() { - Map struct1 = new HashMap<>(); - struct1.put("string", "Test String 1"); - struct1.put("integer", 10); - - Map struct2 = new HashMap<>(); - struct2.put("string", "Test String 2"); - struct2.put("integer", 20); - - return new MapRecord(getRecordInListSchema(), Collections.singletonMap("list", Arrays.asList(struct1, struct2))); - } - - private static Record setupRecordInMapTestRecord() { - Map struct1 = new HashMap<>(); - struct1.put("string", "Test String 1"); - struct1.put("integer", 10); - - Map struct2 = new HashMap<>(); - struct2.put("string", "Test String 2"); - struct2.put("integer", 20); - - Map> map = new HashMap<>(); - map.put("key1", struct1); - map.put("key2", struct2); - - return new MapRecord(getMapSchema(), Collections.singletonMap("map", map)); - } - - private static Record setupPrimitivesTestRecord() { - LocalDate localDate = LocalDate.of(2017, 4, 4); - LocalTime localTime = LocalTime.of(14, 20, 33); - OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5)); - - Map values = new HashMap<>(); - values.put("string", "Test String"); - values.put("integer", 8); - values.put("float", 1.23456F); - values.put("long", 42L); - values.put("double", 3.14159D); - values.put("decimal", new BigDecimal("12345678.12")); - values.put("decimalLowerScore", 12345678.1); - values.put("boolean", true); - values.put("fixed", "hello".getBytes()); - values.put("binary", "hello".getBytes()); - values.put("date", localDate); - values.put("time", Time.valueOf(localTime)); - values.put("timestamp", Timestamp.from(offsetDateTime.toInstant())); - values.put("timestampTz", Timestamp.valueOf(LOCAL_DATE_TIME)); - values.put("uuid", UUID.fromString("0000-00-00-00-000000")); - values.put("choice", "10"); - values.put("enum", "blue"); - - return new MapRecord(getPrimitivesSchema(), values); - } - - private static Record setupPrimitivesTestRecordMissingFields() { - LocalDate localDate = LocalDate.of(2017, 4, 4); - LocalTime localTime = LocalTime.of(14, 20, 33); - OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5)); - - Map values = new HashMap<>(); - values.put("string", "Test String"); - values.put("double", 3.14159D); - values.put("decimal", new BigDecimal("12345678.12")); - values.put("boolean", true); - values.put("fixed", "hello".getBytes()); - values.put("binary", "hello".getBytes()); - values.put("date", localDate); - values.put("time", Time.valueOf(localTime)); - values.put("timestamp", Timestamp.from(offsetDateTime.toInstant())); - values.put("timestampTz", Timestamp.valueOf(LOCAL_DATE_TIME)); - values.put("uuid", UUID.fromString("0000-00-00-00-000000")); - values.put("choice", "10"); - - return new MapRecord(getPrimitivesSchemaMissingFields(), values); - } - - private static Record setupCompatiblePrimitivesTestRecord() { - - Map values = new HashMap<>(); - values.put("string", 123); - values.put("integer", 8); - values.put("float", 1.23456); - values.put("long", 42L); - values.put("double", 3.14159); - values.put("decimal", 12345678.12); - values.put("fixed", "hello".getBytes()); - values.put("binary", "hello".getBytes()); - values.put("date", "2017-04-04"); - values.put("time", "14:20:33.000"); - values.put("timestamp", Timestamp.valueOf(LOCAL_DATE_TIME)); - values.put("timestampTz", Timestamp.valueOf(LOCAL_DATE_TIME)); - values.put("uuid", "0000-00-00-00-000000"); - values.put("choice", "10"); - - return new MapRecord(getPrimitivesAsCompatiblesSchema(), values); - } - - private static Record setupCaseInsensitiveTestRecord() { - Map values = new HashMap<>(); - values.put("field1", "Text1"); - values.put("FIELD2", "Text2"); - values.put("Field3", "Text3"); - values.put("fielD4", "Text4"); - - return new MapRecord(getCaseInsensitiveSchema(), values); - } - - private static Record setupUnorderedTestRecord() { - List listValues = new ArrayList<>(); - listValues.add("list value2"); - listValues.add("list value1"); - - Map mapValues = new HashMap<>(); - mapValues.put("key2", "map value2"); - mapValues.put("key1", "map value1"); - - Map nestedValues = new HashMap<>(); - nestedValues.put("FIELD4", listValues); - nestedValues.put("FIELD3", "value3"); - MapRecord nestedRecord = new MapRecord(getNestedUnorderedSchema(), nestedValues); - - Map values = new HashMap<>(); - values.put("FIELD6", mapValues); - values.put("FIELD5", "value5"); - values.put("FIELD1", "value1"); - values.put("FIELD2", nestedRecord); - return new MapRecord(getStructSchema(), values); - } - - private static Record setupChoiceTestRecord() { - Map values = new HashMap<>(); - values.put("choice1", "20"); - values.put("choice2", "30a"); - values.put("choice3", String.valueOf(Long.MAX_VALUE)); - - return new MapRecord(getChoiceSchema(), values); - } - - @DisabledOnOs(WINDOWS) - @ParameterizedTest - @EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"}) - public void testPrimitives(FileFormat format) throws IOException { - RecordSchema nifiSchema = getPrimitivesSchema(); - Record record = setupPrimitivesTestRecord(); - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(PRIMITIVES_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, PRIMITIVES_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, PRIMITIVES_SCHEMA, tempFile.toInputFile()); - - assertEquals(results.size(), 1); - GenericRecord resultRecord = results.getFirst(); - - OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5)); - - assertEquals("Test String", resultRecord.get(0, String.class)); - assertEquals(Integer.valueOf(8), resultRecord.get(1, Integer.class)); - assertEquals(Float.valueOf(1.23456F), resultRecord.get(2, Float.class)); - assertEquals(Long.valueOf(42L), resultRecord.get(3, Long.class)); - assertEquals(Double.valueOf(3.14159D), resultRecord.get(4, Double.class)); - assertEquals(new BigDecimal("12345678.12"), resultRecord.get(5, BigDecimal.class)); - assertEquals(new BigDecimal("12345678.10"), resultRecord.get(6, BigDecimal.class)); - assertEquals(Boolean.TRUE, resultRecord.get(7, Boolean.class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(8, byte[].class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(9, ByteBuffer.class).array()); - assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(10, LocalDate.class)); - assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(11, LocalTime.class)); - assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(12, OffsetDateTime.class)); - assertEquals(LOCAL_DATE_TIME, resultRecord.get(13, LocalDateTime.class)); - assertEquals(Integer.valueOf(10), resultRecord.get(15, Integer.class)); - assertEquals("blue", resultRecord.get(16, String.class)); - - if (format.equals(PARQUET)) { - assertArrayEquals(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, resultRecord.get(14, byte[].class)); - } else { - assertEquals(UUID.fromString("0000-00-00-00-000000"), resultRecord.get(14, UUID.class)); - } - } - - @DisabledOnOs(WINDOWS) - @ParameterizedTest - @EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"}) - public void testPrimitivesIgnoreMissingFields(FileFormat format) throws IOException { - RecordSchema nifiSchema = getPrimitivesSchemaMissingFields(); - Record record = setupPrimitivesTestRecordMissingFields(); - MockComponentLog mockComponentLogger = new MockComponentLog("id", "TestIcebergRecordConverter"); - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(PRIMITIVES_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, mockComponentLogger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, PRIMITIVES_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, PRIMITIVES_SCHEMA, tempFile.toInputFile()); - - assertEquals(results.size(), 1); - GenericRecord resultRecord = results.getFirst(); - - OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5)); - - assertEquals("Test String", resultRecord.get(0, String.class)); - assertNull(resultRecord.get(1, Integer.class)); - assertNull(resultRecord.get(2, Float.class)); - assertNull(resultRecord.get(3, Long.class)); - assertEquals(Double.valueOf(3.14159D), resultRecord.get(4, Double.class)); - assertEquals(new BigDecimal("12345678.12"), resultRecord.get(5, BigDecimal.class)); - assertEquals(Boolean.TRUE, resultRecord.get(7, Boolean.class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(8, byte[].class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(9, ByteBuffer.class).array()); - assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(10, LocalDate.class)); - assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(11, LocalTime.class)); - assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(12, OffsetDateTime.class)); - assertEquals(LOCAL_DATE_TIME, resultRecord.get(13, LocalDateTime.class)); - assertEquals(Integer.valueOf(10), resultRecord.get(15, Integer.class)); - - if (format.equals(FileFormat.PARQUET)) { - // Parquet uses a conversion to the byte values of numeric characters such as "0" -> byte value 0 - UUID uuid = UUID.fromString("0000-00-00-00-000000"); - ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[16]); - byteBuffer.putLong(uuid.getMostSignificantBits()); - byteBuffer.putLong(uuid.getLeastSignificantBits()); - assertArrayEquals(byteBuffer.array(), resultRecord.get(14, byte[].class)); - } else { - assertEquals(UUID.fromString("0000-00-00-00-000000"), resultRecord.get(14, UUID.class)); - } - - // Test null values - for (String fieldName : record.getRawFieldNames()) { - record.setValue(fieldName, null); - } - - genericRecord = recordConverter.convert(record); - - writeTo(format, PRIMITIVES_SCHEMA, genericRecord, tempFile); - - results = readFrom(format, PRIMITIVES_SCHEMA, tempFile.toInputFile()); - - assertEquals(results.size(), 1); - resultRecord = results.getFirst(); - assertNull(resultRecord.get(0, String.class)); - assertNull(resultRecord.get(1, Integer.class)); - assertNull(resultRecord.get(2, Float.class)); - assertNull(resultRecord.get(3, Long.class)); - assertNull(resultRecord.get(4, Double.class)); - assertNull(resultRecord.get(5, BigDecimal.class)); - assertNull(resultRecord.get(6, Boolean.class)); - assertNull(resultRecord.get(7)); - assertNull(resultRecord.get(8)); - assertNull(resultRecord.get(9, LocalDate.class)); - assertNull(resultRecord.get(10, LocalTime.class)); - assertNull(resultRecord.get(11, OffsetDateTime.class)); - assertNull(resultRecord.get(14, Integer.class)); - } - - @DisabledOnOs(WINDOWS) - @ParameterizedTest - @EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"}) - public void testPrimitivesMissingRequiredFields(FileFormat format) { - RecordSchema nifiSchema = getPrimitivesSchemaMissingFields(); - MockComponentLog mockComponentLogger = new MockComponentLog("id", "TestIcebergRecordConverter"); - - assertThrows(IllegalArgumentException.class, - () -> new IcebergRecordConverter(PRIMITIVES_SCHEMA_WITH_REQUIRED_FIELDS, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, mockComponentLogger)); - } - - @DisabledOnOs(WINDOWS) - @ParameterizedTest - @EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"}) - public void testPrimitivesWarnMissingFields(FileFormat format) throws IOException { - RecordSchema nifiSchema = getPrimitivesSchemaMissingFields(); - Record record = setupPrimitivesTestRecordMissingFields(); - MockComponentLog mockComponentLogger = new MockComponentLog("id", "TestIcebergRecordConverter"); - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(PRIMITIVES_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.WARNING_UNMATCHED_COLUMN, mockComponentLogger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, PRIMITIVES_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, PRIMITIVES_SCHEMA, tempFile.toInputFile()); - - assertEquals(results.size(), 1); - GenericRecord resultRecord = results.getFirst(); - - OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5)); - - assertEquals("Test String", resultRecord.get(0, String.class)); - assertNull(resultRecord.get(1, Integer.class)); - assertNull(resultRecord.get(2, Float.class)); - assertNull(resultRecord.get(3, Long.class)); - assertEquals(Double.valueOf(3.14159D), resultRecord.get(4, Double.class)); - assertEquals(new BigDecimal("12345678.12"), resultRecord.get(5, BigDecimal.class)); - assertEquals(Boolean.TRUE, resultRecord.get(7, Boolean.class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(8, byte[].class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(9, ByteBuffer.class).array()); - assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(10, LocalDate.class)); - assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(11, LocalTime.class)); - assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(12, OffsetDateTime.class)); - assertEquals(LOCAL_DATE_TIME, resultRecord.get(13, LocalDateTime.class)); - assertEquals(Integer.valueOf(10), resultRecord.get(15, Integer.class)); - - if (format.equals(FileFormat.PARQUET)) { - // Parquet uses a conversion to the byte values of numeric characters such as "0" -> byte value 0 - UUID uuid = UUID.fromString("0000-00-00-00-000000"); - ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[16]); - byteBuffer.putLong(uuid.getMostSignificantBits()); - byteBuffer.putLong(uuid.getLeastSignificantBits()); - assertArrayEquals(byteBuffer.array(), resultRecord.get(14, byte[].class)); - } else { - assertEquals(UUID.fromString("0000-00-00-00-000000"), resultRecord.get(14, UUID.class)); - } - } - - @DisabledOnOs(WINDOWS) - @ParameterizedTest - @EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"}) - public void testPrimitivesFailMissingFields(FileFormat format) { - RecordSchema nifiSchema = getPrimitivesSchemaMissingFields(); - MockComponentLog mockComponentLogger = new MockComponentLog("id", "TestIcebergRecordConverter"); - - assertThrows(IllegalArgumentException.class, - () -> new IcebergRecordConverter(PRIMITIVES_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.FAIL_UNMATCHED_COLUMN, mockComponentLogger)); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testCompatiblePrimitives() throws IOException { - RecordSchema nifiSchema = getPrimitivesAsCompatiblesSchema(); - Record record = setupCompatiblePrimitivesTestRecord(); - final FileFormat format = PARQUET; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(COMPATIBLE_PRIMITIVES_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.FAIL_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, COMPATIBLE_PRIMITIVES_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, COMPATIBLE_PRIMITIVES_SCHEMA, tempFile.toInputFile()); - - assertEquals(results.size(), 1); - GenericRecord resultRecord = results.getFirst(); - - final ZonedDateTime zonedDateTime = ZonedDateTime.of(LOCAL_DATE_TIME, ZoneId.systemDefault()); - OffsetDateTime offsetDateTime = zonedDateTime.toOffsetDateTime(); - - assertEquals("123", resultRecord.get(0, String.class)); - assertEquals(Integer.valueOf(8), resultRecord.get(1, Integer.class)); - assertEquals(Float.valueOf(1.23456F), resultRecord.get(2, Float.class)); - assertEquals(Long.valueOf(42L), resultRecord.get(3, Long.class)); - assertEquals(Double.valueOf(3.141590118408203), resultRecord.get(4, Double.class)); - assertEquals(new BigDecimal("12345678.12"), resultRecord.get(5, BigDecimal.class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(6, byte[].class)); - assertArrayEquals(new byte[]{104, 101, 108, 108, 111}, resultRecord.get(7, ByteBuffer.class).array()); - assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(8, LocalDate.class)); - assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(9, LocalTime.class)); - assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(10, OffsetDateTime.class)); - assertEquals(LOCAL_DATE_TIME, resultRecord.get(11, LocalDateTime.class)); - assertEquals(Integer.valueOf(10), resultRecord.get(13, Integer.class)); - - assertArrayEquals(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, resultRecord.get(12, byte[].class)); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testStruct() throws IOException { - RecordSchema nifiSchema = getStructSchema(); - Record record = setupStructTestRecord(); - final FileFormat format = FileFormat.ORC; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(STRUCT_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, STRUCT_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, STRUCT_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - GenericRecord resultRecord = results.getFirst(); - - assertEquals(1, resultRecord.size()); - assertInstanceOf(GenericRecord.class, resultRecord.get(0)); - GenericRecord nestedRecord = (GenericRecord) resultRecord.get(0); - - assertEquals(1, nestedRecord.size()); - assertInstanceOf(GenericRecord.class, nestedRecord.get(0)); - GenericRecord baseRecord = (GenericRecord) nestedRecord.get(0); - - assertEquals("Test String", baseRecord.get(0, String.class)); - assertEquals(Integer.valueOf(10), baseRecord.get(1, Integer.class)); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testList() throws IOException { - RecordSchema nifiSchema = getListSchema(); - Record record = setupListTestRecord(); - final FileFormat format = FileFormat.AVRO; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(LIST_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, LIST_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, LIST_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - GenericRecord resultRecord = results.getFirst(); - - assertEquals(1, resultRecord.size()); - assertInstanceOf(List.class, resultRecord.get(0)); - List nestedList = resultRecord.get(0, List.class); - - assertEquals(2, nestedList.size()); - assertInstanceOf(List.class, nestedList.get(0)); - assertInstanceOf(List.class, nestedList.get(1)); - - assertTrue(((List) nestedList.get(0)).containsAll(List.of("Test String1", "Test String2"))); - assertTrue(((List) nestedList.get(1)).containsAll(List.of("Test String3", "Test String4"))); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testMap() throws IOException { - RecordSchema nifiSchema = getMapSchema(); - Record record = setupMapTestRecord(); - final FileFormat format = PARQUET; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(MAP_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, MAP_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, MAP_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - GenericRecord resultRecord = results.getFirst(); - - assertEquals(1, resultRecord.size()); - assertInstanceOf(Map.class, resultRecord.get(0)); - Map nestedMap = resultRecord.get(0, Map.class); - - assertEquals(1, nestedMap.size()); - assertInstanceOf(Map.class, nestedMap.get("key")); - Map baseMap = (Map) nestedMap.get("key"); - - assertEquals(42L, baseMap.get("nested_key")); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testRecordInList() throws IOException { - RecordSchema nifiSchema = getRecordInListSchema(); - Record record = setupRecordInListTestRecord(); - final FileFormat format = FileFormat.AVRO; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(RECORD_IN_LIST_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, RECORD_IN_LIST_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, RECORD_IN_LIST_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - assertInstanceOf(GenericRecord.class, results.get(0)); - GenericRecord resultRecord = results.get(0); - - assertEquals(1, resultRecord.size()); - assertInstanceOf(List.class, resultRecord.get(0)); - List fieldList = resultRecord.get(0, List.class); - - assertEquals(2, fieldList.size()); - assertInstanceOf(GenericRecord.class, fieldList.get(0)); - assertInstanceOf(GenericRecord.class, fieldList.get(1)); - - GenericRecord record1 = (GenericRecord) fieldList.get(0); - GenericRecord record2 = (GenericRecord) fieldList.get(1); - - assertEquals("Test String 1", record1.get(0, String.class)); - assertEquals(Integer.valueOf(10), record1.get(1, Integer.class)); - - assertEquals("Test String 2", record2.get(0, String.class)); - assertEquals(Integer.valueOf(20), record2.get(1, Integer.class)); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testRecordInMap() throws IOException { - RecordSchema nifiSchema = getRecordInMapSchema(); - Record record = setupRecordInMapTestRecord(); - final FileFormat format = FileFormat.ORC; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(RECORD_IN_MAP_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, RECORD_IN_MAP_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, RECORD_IN_MAP_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - assertInstanceOf(GenericRecord.class, results.get(0)); - GenericRecord resultRecord = results.get(0); - - assertEquals(1, resultRecord.size()); - assertInstanceOf(Map.class, resultRecord.get(0)); - Map recordMap = resultRecord.get(0, Map.class); - - assertEquals(2, recordMap.size()); - assertInstanceOf(GenericRecord.class, recordMap.get("key1")); - assertInstanceOf(GenericRecord.class, recordMap.get("key2")); - - GenericRecord record1 = (GenericRecord) recordMap.get("key1"); - GenericRecord record2 = (GenericRecord) recordMap.get("key2"); - - assertEquals("Test String 1", record1.get(0, String.class)); - assertEquals(Integer.valueOf(10), record1.get(1, Integer.class)); - - assertEquals("Test String 2", record2.get(0, String.class)); - assertEquals(Integer.valueOf(20), record2.get(1, Integer.class)); - } - - @DisabledOnOs(WINDOWS) - @ParameterizedTest - @EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"}) - public void testSchemaMismatch(FileFormat format) { - RecordSchema nifiSchema = getPrimitivesSchemaMissingFields(); - - IllegalArgumentException e = assertThrows(IllegalArgumentException.class, - () -> new IcebergRecordConverter(PRIMITIVES_SCHEMA_WITH_REQUIRED_FIELDS, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger)); - assertTrue(e.getMessage().contains("Iceberg requires a non-null value for required fields"), e.getMessage()); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testCaseInsensitiveFieldMapping() throws IOException { - RecordSchema nifiSchema = getCaseInsensitiveSchema(); - Record record = setupCaseInsensitiveTestRecord(); - final FileFormat format = FileFormat.AVRO; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(CASE_INSENSITIVE_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, CASE_INSENSITIVE_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, CASE_INSENSITIVE_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - GenericRecord resultRecord = results.getFirst(); - - assertEquals("Text1", resultRecord.get(0, String.class)); - assertEquals("Text2", resultRecord.get(1, String.class)); - assertEquals("Text3", resultRecord.get(2, String.class)); - assertEquals("Text4", resultRecord.get(3, String.class)); - } - - @DisabledOnOs(WINDOWS) - @Test - public void testUnorderedFieldMapping() throws IOException { - RecordSchema nifiSchema = getUnorderedSchema(); - Record record = setupUnorderedTestRecord(); - final FileFormat format = PARQUET; - - IcebergRecordConverter recordConverter = new IcebergRecordConverter(UNORDERED_SCHEMA, nifiSchema, format, UnmatchedColumnBehavior.IGNORE_UNMATCHED_COLUMN, logger); - GenericRecord genericRecord = recordConverter.convert(record); - - writeTo(format, UNORDERED_SCHEMA, genericRecord, tempFile); - - List results = readFrom(format, UNORDERED_SCHEMA, tempFile.toInputFile()); - - assertEquals(1, results.size()); - GenericRecord resultRecord = results.getFirst(); - - assertEquals("value1", resultRecord.get(0, String.class)); - - assertInstanceOf(GenericRecord.class, resultRecord.get(1)); - GenericRecord nestedRecord = (GenericRecord) resultRecord.get(1); - - assertEquals("value3", nestedRecord.get(0, String.class)); - - assertInstanceOf(List.class, nestedRecord.get(1)); - List nestedList = nestedRecord.get(1, List.class); - assertTrue(nestedList.containsAll(List.of("list value1", "list value2"))); - - assertEquals("value5", resultRecord.get(2, String.class)); - - assertInstanceOf(Map.class, resultRecord.get(3)); - Map map = resultRecord.get(3, Map.class); - assertEquals("map value1", map.get("key1")); - assertEquals("map value2", map.get("key2")); - } - - @Test - public void testChoiceDataTypeInRecord() { - Record record = setupChoiceTestRecord(); - DataType dataType = RecordFieldType.CHOICE.getChoiceDataType( - RecordFieldType.STRING.getDataType(), RecordFieldType.INT.getDataType(), RecordFieldType.LONG.getDataType()); - - RecordFieldGetter.FieldGetter fieldGetter1 = RecordFieldGetter.createFieldGetter(dataType, "choice1", true); - RecordFieldGetter.FieldGetter fieldGetter2 = RecordFieldGetter.createFieldGetter(dataType, "choice2", true); - RecordFieldGetter.FieldGetter fieldGetter3 = RecordFieldGetter.createFieldGetter(dataType, "choice3", true); - - assertInstanceOf(Integer.class, fieldGetter1.getFieldOrNull(record)); - assertInstanceOf(String.class, fieldGetter2.getFieldOrNull(record)); - assertInstanceOf(Long.class, fieldGetter3.getFieldOrNull(record)); - } - - @Test - public void testChoiceDataTypeInArray() { - DataType dataType = RecordFieldType.CHOICE.getChoiceDataType( - RecordFieldType.STRING.getDataType(), RecordFieldType.INT.getDataType(), RecordFieldType.LONG.getDataType()); - ArrayElementGetter.ElementGetter elementGetter = ArrayElementGetter.createElementGetter(dataType); - - String[] testArray = {"20", "30a", String.valueOf(Long.MAX_VALUE)}; - - assertInstanceOf(Integer.class, elementGetter.getElementOrNull(testArray[0])); - assertInstanceOf(String.class, elementGetter.getElementOrNull(testArray[1])); - assertInstanceOf(Long.class, elementGetter.getElementOrNull(testArray[2])); - } - - private void writeTo(FileFormat format, Schema schema, GenericRecord record, OutputFile outputFile) throws IOException { - switch (format) { - case AVRO: - writeToAvro(schema, record, outputFile); - break; - case ORC: - writeToOrc(schema, record, outputFile); - break; - case PARQUET: - writeToParquet(schema, record, outputFile); - break; - } - } - - private ArrayList readFrom(FileFormat format, Schema schema, InputFile inputFile) throws IOException { - return switch (format) { - case AVRO -> readFromAvro(schema, inputFile); - case ORC -> readFromOrc(schema, inputFile); - case PARQUET -> readFromParquet(schema, inputFile); - default -> throw new IOException("Unknown file format: " + format); - }; - } - - private void writeToAvro(Schema schema, GenericRecord record, OutputFile outputFile) throws IOException { - try (FileAppender appender = Avro.write(outputFile) - .schema(schema) - .createWriterFunc(DataWriter::create) - .overwrite() - .build()) { - appender.add(record); - } - } - - private ArrayList readFromAvro(Schema schema, InputFile inputFile) throws IOException { - try (AvroIterable reader = Avro.read(inputFile) - .project(schema) - .createReaderFunc(DataReader::create) - .build()) { - return Lists.newArrayList(reader); - } - } - - private void writeToOrc(Schema schema, GenericRecord record, OutputFile outputFile) throws IOException { - try (FileAppender appender = ORC.write(outputFile) - .schema(schema) - .createWriterFunc(GenericOrcWriter::buildWriter) - .overwrite() - .build()) { - appender.add(record); - } - } - - private ArrayList readFromOrc(Schema schema, InputFile inputFile) throws IOException { - try (CloseableIterable reader = ORC.read(inputFile) - .project(schema) - .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(schema, fileSchema)) - .build()) { - return Lists.newArrayList(reader); - } - } - - private void writeToParquet(Schema schema, GenericRecord record, OutputFile outputFile) throws IOException { - try (FileAppender appender = Parquet.write(outputFile) - .schema(schema) - .createWriterFunc(GenericParquetWriter::buildWriter) - .overwrite() - .build()) { - appender.add(record); - } - } - - private ArrayList readFromParquet(Schema schema, InputFile inputFile) throws IOException { - try (CloseableIterable reader = Parquet.read(inputFile) - .project(schema) - .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) - .build()) { - return Lists.newArrayList(reader); - } - } -} \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergCustomValidation.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergCustomValidation.java deleted file mode 100644 index f1ab1d3983..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergCustomValidation.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.nifi.kerberos.KerberosUserService; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.serialization.record.MockRecordParser; -import org.apache.nifi.services.iceberg.HiveCatalogService; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import static org.apache.nifi.services.iceberg.AbstractCatalogService.HADOOP_CONFIGURATION_RESOURCES; -import static org.apache.nifi.services.iceberg.HiveCatalogService.METASTORE_URI; -import static org.apache.nifi.services.iceberg.HiveCatalogService.WAREHOUSE_LOCATION; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class TestPutIcebergCustomValidation { - - private static final String RECORD_READER_NAME = "record-reader"; - private static final String KERBEROS_USER_SERVICE_NAME = "kerberos-user-service"; - private static final String CATALOG_SERVICE_NAME = "catalog-service"; - - private static final String CATALOG_NAMESPACE = "catalogNamespace"; - private static final String TABLE_NAME = "tableName"; - - private TestRunner runner; - - @BeforeEach - public void setUp() { - PutIceberg processor = new PutIceberg(); - runner = TestRunners.newTestRunner(processor); - } - - private void initRecordReader() throws InitializationException { - MockRecordParser readerFactory = new MockRecordParser(); - - runner.addControllerService(RECORD_READER_NAME, readerFactory); - runner.enableControllerService(readerFactory); - - runner.setProperty(PutIceberg.RECORD_READER, RECORD_READER_NAME); - } - - private void initCatalogService(String configFilePaths) throws InitializationException { - final HiveCatalogService catalogService = new HiveCatalogService(); - runner.addControllerService(CATALOG_SERVICE_NAME, catalogService); - runner.setProperty(catalogService, METASTORE_URI, "test-metastore"); - runner.setProperty(catalogService, WAREHOUSE_LOCATION, "test-warehouse"); - runner.setProperty(catalogService, HADOOP_CONFIGURATION_RESOURCES, configFilePaths); - - runner.enableControllerService(catalogService); - - runner.setProperty(PutIceberg.CATALOG, CATALOG_SERVICE_NAME); - } - - private void initKerberosUserService() throws InitializationException { - KerberosUserService kerberosUserService = mock(KerberosUserService.class); - when(kerberosUserService.getIdentifier()).thenReturn(KERBEROS_USER_SERVICE_NAME); - - runner.addControllerService(KERBEROS_USER_SERVICE_NAME, kerberosUserService); - runner.enableControllerService(kerberosUserService); - - runner.setProperty(PutIceberg.KERBEROS_USER_SERVICE, KERBEROS_USER_SERVICE_NAME); - } - - @Test - public void testCustomValidateWithKerberosSecurityConfigAndWithoutKerberosUserService() throws InitializationException { - initRecordReader(); - initCatalogService("src/test/resources/secured-core-site.xml"); - - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAMESPACE); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.assertNotValid(); - } - - @Test - public void testCustomValidateWithKerberosSecurityConfigAndKerberosUserService() throws InitializationException { - initRecordReader(); - initCatalogService("src/test/resources/secured-core-site.xml"); - - initKerberosUserService(); - - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAMESPACE); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.assertValid(); - } - - @Test - public void testCustomValidateWithoutKerberosSecurityConfigAndKerberosUserService() throws InitializationException { - initRecordReader(); - initCatalogService("src/test/resources/unsecured-core-site.xml"); - - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAMESPACE); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.assertValid(); - } - - @Test - public void testCustomValidateWithoutKerberosSecurityConfigAndWithKerberosUserService() throws InitializationException { - initRecordReader(); - initCatalogService("src/test/resources/unsecured-core-site.xml"); - - initKerberosUserService(); - - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAMESPACE); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.assertNotValid(); - } - - @Test - public void testInvalidSnapshotSummaryDynamicProperty() throws InitializationException { - initRecordReader(); - initCatalogService("src/test/resources/unsecured-core-site.xml"); - - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAMESPACE); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - - runner.setProperty("invalid.dynamic.property", "test value"); - runner.assertNotValid(); - } - - @Test - public void testValidSnapshotSummaryDynamicProperty() throws InitializationException { - initRecordReader(); - initCatalogService("src/test/resources/unsecured-core-site.xml"); - - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAMESPACE); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - - runner.setProperty("snapshot-property.valid-property", "test value"); - runner.assertValid(); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHadoopCatalog.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHadoopCatalog.java deleted file mode 100644 index 6d120ea2b5..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHadoopCatalog.java +++ /dev/null @@ -1,216 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.avro.Schema; -import org.apache.commons.io.IOUtils; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; -import org.apache.nifi.avro.AvroTypeUtil; -import org.apache.nifi.processors.iceberg.catalog.IcebergCatalogFactory; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.serialization.record.MockRecordParser; -import org.apache.nifi.serialization.record.RecordField; -import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.services.iceberg.HadoopCatalogService; -import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.DisabledOnOs; - -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.nifi.processors.iceberg.PutIceberg.ICEBERG_RECORD_COUNT; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.CATALOG_NAME; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.CATALOG_SERVICE; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.NAMESPACE; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.RECORD_READER_SERVICE; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.createTemporaryDirectory; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validateNumberOfDataFiles; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validatePartitionFolders; -import static org.apache.nifi.services.iceberg.HadoopCatalogService.WAREHOUSE_PATH; -import static org.junit.jupiter.api.condition.OS.WINDOWS; - -public class TestPutIcebergWithHadoopCatalog { - - protected static final String TABLE_NAME = "date_test"; - - private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(NAMESPACE, TABLE_NAME); - - private static final org.apache.iceberg.Schema DATE_SCHEMA = new org.apache.iceberg.Schema( - Types.NestedField.required(1, "timeMicros", Types.TimeType.get()), - Types.NestedField.required(2, "timestampMicros", Types.TimestampType.withZone()), - Types.NestedField.required(3, "date", Types.DateType.get()) - ); - - private TestRunner runner; - private PutIceberg processor; - private static Schema inputSchema; - private Catalog catalog; - - - @BeforeAll - public static void initSchema() throws IOException { - final String avroSchema = IOUtils.toString(Files.newInputStream(Paths.get("src/test/resources/date.avsc")), StandardCharsets.UTF_8); - inputSchema = new Schema.Parser().parse(avroSchema); - } - - @BeforeEach - public void setUp() { - processor = new PutIceberg(); - } - - private void initRecordReader() throws InitializationException { - final MockRecordParser readerFactory = new MockRecordParser(); - final RecordSchema recordSchema = AvroTypeUtil.createSchema(inputSchema); - - for (RecordField recordField : recordSchema.getFields()) { - readerFactory.addSchemaField(recordField.getFieldName(), recordField.getDataType().getFieldType(), recordField.isNullable()); - } - - readerFactory.addRecord(Time.valueOf("15:30:30"), Timestamp.valueOf("2015-02-02 15:30:30.800"), Date.valueOf("2015-02-02")); - readerFactory.addRecord(Time.valueOf("15:30:30"), Timestamp.valueOf("2015-02-02 15:30:30.800"), Date.valueOf("2015-02-02")); - readerFactory.addRecord(Time.valueOf("15:30:30"), Timestamp.valueOf("2016-01-02 15:30:30.800"), Date.valueOf("2016-01-02")); - readerFactory.addRecord(Time.valueOf("15:30:30"), Timestamp.valueOf("2017-01-10 15:30:30.800"), Date.valueOf("2017-01-10")); - - runner.addControllerService(RECORD_READER_SERVICE, readerFactory); - runner.enableControllerService(readerFactory); - - runner.setProperty(PutIceberg.RECORD_READER, RECORD_READER_SERVICE); - } - - private void initCatalog(PartitionSpec spec, FileFormat fileFormat) throws InitializationException { - final File warehousePath = createTemporaryDirectory(); - final HadoopCatalogService catalogService = new HadoopCatalogService(); - runner.addControllerService(CATALOG_SERVICE, catalogService); - runner.setProperty(catalogService, WAREHOUSE_PATH, warehousePath.getAbsolutePath()); - runner.enableControllerService(catalogService); - - final IcebergCatalogFactory catalogFactory = new IcebergCatalogFactory(catalogService); - catalog = catalogFactory.create(); - - final Map tableProperties = new HashMap<>(); - tableProperties.put(TableProperties.DEFAULT_FILE_FORMAT, fileFormat.name()); - - catalog.createTable(TABLE_IDENTIFIER, DATE_SCHEMA, spec, tableProperties); - - runner.setProperty(PutIceberg.CATALOG, CATALOG_SERVICE); - } - - @DisabledOnOs(WINDOWS) - @Test - public void onTriggerYearTransform() throws Exception { - PartitionSpec spec = PartitionSpec.builderFor(DATE_SCHEMA) - .year("date") - .build(); - - runner = TestRunners.newTestRunner(processor); - initRecordReader(); - initCatalog(spec, FileFormat.PARQUET); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAME); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.setValidateExpressionUsage(false); - runner.enqueue(new byte[0]); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - Assertions.assertTrue(table.spec().isPartitioned()); - Assertions.assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateNumberOfDataFiles(table.location(), 3); - validatePartitionFolders(table.location(), Arrays.asList("date_year=2015", "date_year=2016", "date_year=2017")); - } - - @DisabledOnOs(WINDOWS) - @Test - public void onTriggerMonthTransform() throws Exception { - final PartitionSpec spec = PartitionSpec.builderFor(DATE_SCHEMA) - .month("timestampMicros") - .build(); - - runner = TestRunners.newTestRunner(processor); - initRecordReader(); - initCatalog(spec, FileFormat.ORC); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAME); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.setValidateExpressionUsage(false); - runner.enqueue(new byte[0]); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - Assertions.assertTrue(table.spec().isPartitioned()); - Assertions.assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateNumberOfDataFiles(table.location(), 3); - validatePartitionFolders(table.location(), Arrays.asList( - "timestampMicros_month=2015-02", "timestampMicros_month=2016-01", "timestampMicros_month=2017-01")); - } - - @DisabledOnOs(WINDOWS) - @Test - public void onTriggerDayTransform() throws Exception { - final PartitionSpec spec = PartitionSpec.builderFor(DATE_SCHEMA) - .day("timestampMicros") - .build(); - - runner = TestRunners.newTestRunner(processor); - initRecordReader(); - initCatalog(spec, FileFormat.AVRO); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAME); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.setValidateExpressionUsage(false); - runner.enqueue(new byte[0]); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - Assertions.assertTrue(table.spec().isPartitioned()); - Assertions.assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateNumberOfDataFiles(table.location(), 3); - validatePartitionFolders(table.location(), Arrays.asList( - "timestampMicros_day=2015-02-02", "timestampMicros_day=2016-01-02", "timestampMicros_day=2017-01-10")); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHiveCatalog.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHiveCatalog.java deleted file mode 100644 index 21ff765c1f..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithHiveCatalog.java +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.nifi.hive.metastore.ThriftMetastore; -import org.apache.nifi.processors.iceberg.catalog.IcebergCatalogFactory; -import org.apache.nifi.processors.iceberg.util.IcebergTestUtils; -import org.apache.nifi.provenance.ProvenanceEventRecord; -import org.apache.nifi.provenance.ProvenanceEventType; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.services.iceberg.HiveCatalogService; -import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.TestRunners; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.DisabledOnOs; -import org.junit.jupiter.api.extension.RegisterExtension; - -import java.net.URI; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.nifi.processors.iceberg.PutIceberg.ICEBERG_RECORD_COUNT; -import static org.apache.nifi.processors.iceberg.PutIceberg.ICEBERG_SNAPSHOT_SUMMARY_FLOWFILE_UUID; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.CATALOG_NAME; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validateData; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validateNumberOfDataFiles; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validatePartitionFolders; -import static org.apache.nifi.services.iceberg.HiveCatalogService.METASTORE_URI; -import static org.apache.nifi.services.iceberg.HiveCatalogService.WAREHOUSE_LOCATION; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.condition.OS.WINDOWS; - -@DisabledOnOs(WINDOWS) -public class TestPutIcebergWithHiveCatalog extends AbstractTestPutIceberg { - - @RegisterExtension - public static ThriftMetastore metastore = new ThriftMetastore(); - - private void initCatalog(PartitionSpec spec, FileFormat fileFormat) throws InitializationException { - final Map tableProperties = new HashMap<>(); - tableProperties.put(TableProperties.DEFAULT_FILE_FORMAT, fileFormat.name()); - - final HiveCatalogService catalogService = new HiveCatalogService(); - runner.addControllerService("catalog-service", catalogService); - runner.setProperty(catalogService, METASTORE_URI, metastore.getThriftConnectionUri()); - runner.setProperty(catalogService, WAREHOUSE_LOCATION, warehousePath); - runner.enableControllerService(catalogService); - - final IcebergCatalogFactory catalogFactory = new IcebergCatalogFactory(catalogService); - catalog = catalogFactory.create(); - - catalog.createTable(TABLE_IDENTIFIER, USER_SCHEMA, spec, tableProperties); - - runner.setProperty(PutIceberg.CATALOG, "catalog-service"); - } - - @Test - public void onTriggerIdentityPartitioned() throws Exception { - final PartitionSpec spec = PartitionSpec.builderFor(USER_SCHEMA) - .identity("department") - .build(); - - runner = TestRunners.newTestRunner(processor); - initRecordReader(); - initCatalog(spec, FileFormat.ORC); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAME); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.setValidateExpressionUsage(false); - runner.enqueue(new byte[0]); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - final List expectedRecords = IcebergTestUtils.RecordsBuilder.newInstance(USER_SCHEMA) - .add(0, "John", "Finance") - .add(1, "Jill", "Finance") - .add(2, "James", "Marketing") - .add(3, "Joana", "Sales") - .build(); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - final String tableLocation = new URI(table.location()).getPath(); - assertTrue(table.spec().isPartitioned()); - assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateData(table, expectedRecords, 0); - validateNumberOfDataFiles(tableLocation, 3); - validatePartitionFolders(tableLocation, Arrays.asList( - "department=Finance", "department=Marketing", "department=Sales")); - assertProvenanceEvents(); - } - - @Test - public void onTriggerMultiLevelIdentityPartitioned() throws Exception { - final PartitionSpec spec = PartitionSpec.builderFor(USER_SCHEMA) - .identity("name") - .identity("department") - .build(); - - runner = TestRunners.newTestRunner(processor); - initRecordReader(); - initCatalog(spec, FileFormat.PARQUET); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAME); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.setValidateExpressionUsage(false); - runner.enqueue(new byte[0]); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - final List expectedRecords = IcebergTestUtils.RecordsBuilder.newInstance(USER_SCHEMA) - .add(0, "John", "Finance") - .add(1, "Jill", "Finance") - .add(2, "James", "Marketing") - .add(3, "Joana", "Sales") - .build(); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - final String tableLocation = new URI(table.location()).getPath(); - assertTrue(table.spec().isPartitioned()); - assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateData(table, expectedRecords, 0); - validateNumberOfDataFiles(tableLocation, 4); - validatePartitionFolders(tableLocation, Arrays.asList( - "name=James/department=Marketing/", - "name=Jill/department=Finance/", - "name=Joana/department=Sales/", - "name=John/department=Finance/" - )); - assertProvenanceEvents(); - } - - @Test - public void onTriggerUnPartitioned() throws Exception { - runner = TestRunners.newTestRunner(processor); - initRecordReader(); - initCatalog(PartitionSpec.unpartitioned(), FileFormat.AVRO); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, "${catalog.name}"); - runner.setProperty(PutIceberg.TABLE_NAME, "${table.name}"); - runner.setProperty(PutIceberg.MAXIMUM_FILE_SIZE, "${max.filesize}"); - runner.setProperty("snapshot-property.additional-summary-property", "test summary property"); - final Map attributes = new HashMap<>(); - attributes.put("catalog.name", CATALOG_NAME); - attributes.put("table.name", TABLE_NAME); - attributes.put("max.filesize", "536870912"); // 512 MB - runner.enqueue(new byte[0], attributes); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - final List expectedRecords = IcebergTestUtils.RecordsBuilder.newInstance(USER_SCHEMA) - .add(0, "John", "Finance") - .add(1, "Jill", "Finance") - .add(2, "James", "Marketing") - .add(3, "Joana", "Sales") - .build(); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - assertTrue(table.spec().isUnpartitioned()); - assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateData(table, expectedRecords, 0); - validateNumberOfDataFiles(new URI(table.location()).getPath(), 1); - assertProvenanceEvents(); - assertSnapshotSummaryProperties(table, Collections.singletonMap("additional-summary-property", "test summary property")); - } - - private void assertProvenanceEvents() { - final List provenanceEvents = runner.getProvenanceEvents(); - assertEquals(1, provenanceEvents.size()); - final ProvenanceEventRecord sendEvent = provenanceEvents.getFirst(); - assertEquals(ProvenanceEventType.SEND, sendEvent.getEventType()); - assertTrue(sendEvent.getTransitUri().endsWith(CATALOG_NAME + ".db/" + TABLE_NAME)); - } - - private void assertSnapshotSummaryProperties(Table table, Map summaryProperties) { - final Map snapshotSummary = table.currentSnapshot().summary(); - - assertTrue(snapshotSummary.containsKey(ICEBERG_SNAPSHOT_SUMMARY_FLOWFILE_UUID)); - - for (Map.Entry entry : summaryProperties.entrySet()) { - assertEquals(snapshotSummary.get(entry.getKey()), entry.getValue()); - } - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithJdbcCatalog.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithJdbcCatalog.java deleted file mode 100644 index c03aff304b..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/TestPutIcebergWithJdbcCatalog.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg; - -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.jdbc.JdbcCatalog; -import org.apache.iceberg.jdbc.JdbcClientPool; -import org.apache.nifi.dbcp.DBCPConnectionPool; -import org.apache.nifi.processors.iceberg.catalog.IcebergJdbcClientPool; -import org.apache.nifi.processors.iceberg.util.IcebergTestUtils; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.services.iceberg.JdbcCatalogService; -import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.TestRunners; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.DisabledOnOs; - -import java.io.File; -import java.net.URI; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.Function; - -import static org.apache.nifi.dbcp.utils.DBCPProperties.DATABASE_URL; -import static org.apache.nifi.dbcp.utils.DBCPProperties.DB_DRIVERNAME; -import static org.apache.nifi.dbcp.utils.DBCPProperties.DB_PASSWORD; -import static org.apache.nifi.dbcp.utils.DBCPProperties.DB_USER; -import static org.apache.nifi.processors.iceberg.PutIceberg.ICEBERG_RECORD_COUNT; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.CATALOG_NAME; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.CATALOG_SERVICE; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.createTemporaryDirectory; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.getSystemTemporaryDirectory; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validateData; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validateNumberOfDataFiles; -import static org.apache.nifi.processors.iceberg.util.IcebergTestUtils.validatePartitionFolders; -import static org.apache.nifi.services.iceberg.JdbcCatalogService.CONNECTION_POOL; -import static org.apache.nifi.services.iceberg.JdbcCatalogService.WAREHOUSE_PATH; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.condition.OS.WINDOWS; - -@DisabledOnOs(WINDOWS) -public class TestPutIcebergWithJdbcCatalog extends AbstractTestPutIceberg { - - private static final String DERBY_LOG_PROPERTY = "derby.stream.error.file"; - private static final String CONNECTION_POOL_SERVICE = "connection-pool-service"; - - private static DBCPConnectionPool connectionPool; - - @BeforeAll - public static void initConnectionPool() { - setDerbyLog(); - connectionPool = new DBCPConnectionPool(); - } - - @AfterAll - public static void clearDerbyLog() { - System.clearProperty(DERBY_LOG_PROPERTY); - } - - private void initServices(PartitionSpec spec) throws InitializationException { - initDBCPService(); - initRecordReader(); - initCatalogService(); - createTestTable(spec); - } - - private void initCatalogService() throws InitializationException { - final JdbcCatalogService catalogService = new JdbcCatalogService(); - runner.addControllerService(CATALOG_SERVICE, catalogService); - runner.setProperty(catalogService, CONNECTION_POOL, CONNECTION_POOL_SERVICE); - runner.setProperty(catalogService, WAREHOUSE_PATH, warehousePath); - runner.enableControllerService(catalogService); - runner.setProperty(PutIceberg.CATALOG, CATALOG_SERVICE); - } - - private void initDBCPService() throws InitializationException { - final String url = String.format("jdbc:derby:%s;create=true", createTemporaryDirectory()); - runner.addControllerService(CONNECTION_POOL_SERVICE, connectionPool); - runner.setProperty(connectionPool, DATABASE_URL, url); - runner.setProperty(connectionPool, DB_USER, String.class.getSimpleName()); - runner.setProperty(connectionPool, DB_PASSWORD, String.class.getName()); - runner.setProperty(connectionPool, DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver"); - runner.enableControllerService(connectionPool); - } - - private void createTestTable(PartitionSpec spec) { - final Map properties = new HashMap<>(); - properties.put(CatalogProperties.URI, ""); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehousePath); - - final Function, JdbcClientPool> clientPoolBuilder = props -> new IcebergJdbcClientPool(props, connectionPool); - final Function, FileIO> ioBuilder = props -> CatalogUtil.loadFileIO("org.apache.iceberg.hadoop.HadoopFileIO", props, new Configuration()); - - catalog = new JdbcCatalog(ioBuilder, clientPoolBuilder, true); - catalog.initialize("jdbc-catalog", properties); - - final Map tableProperties = new HashMap<>(); - tableProperties.put(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name()); - - catalog.createTable(TABLE_IDENTIFIER, USER_SCHEMA, spec, tableProperties); - } - - public static void setDerbyLog() { - final File derbyLog = new File(getSystemTemporaryDirectory(), "derby.log"); - derbyLog.deleteOnExit(); - System.setProperty(DERBY_LOG_PROPERTY, derbyLog.getAbsolutePath()); - } - - @Test - public void onTriggerBucketPartitioned() throws Exception { - final PartitionSpec spec = PartitionSpec.builderFor(USER_SCHEMA) - .bucket("department", 3) - .build(); - - runner = TestRunners.newTestRunner(processor); - runner.setValidateExpressionUsage(false); - initServices(spec); - runner.setProperty(PutIceberg.CATALOG_NAMESPACE, CATALOG_NAME); - runner.setProperty(PutIceberg.TABLE_NAME, TABLE_NAME); - runner.enqueue(new byte[0]); - runner.run(); - - final Table table = catalog.loadTable(TABLE_IDENTIFIER); - - final List expectedRecords = IcebergTestUtils.RecordsBuilder.newInstance(USER_SCHEMA) - .add(0, "John", "Finance") - .add(1, "Jill", "Finance") - .add(2, "James", "Marketing") - .add(3, "Joana", "Sales") - .build(); - - runner.assertTransferCount(PutIceberg.REL_SUCCESS, 1); - final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutIceberg.REL_SUCCESS).getFirst(); - - final String tableLocation = new URI(table.location()).getPath(); - assertTrue(table.spec().isPartitioned()); - assertEquals("4", flowFile.getAttribute(ICEBERG_RECORD_COUNT)); - validateData(table, expectedRecords, 0); - validateNumberOfDataFiles(tableLocation, 3); - validatePartitionFolders(tableLocation, Arrays.asList( - "department_bucket=0", "department_bucket=1", "department_bucket=2")); - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/util/IcebergTestUtils.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/util/IcebergTestUtils.java deleted file mode 100644 index 06918893ac..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/java/org/apache/nifi/processors/iceberg/util/IcebergTestUtils.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.iceberg.util; - -import com.google.common.collect.Lists; -import org.apache.commons.lang.Validate; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.CloseableIterable; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.UUID; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; - -public class IcebergTestUtils { - - public static final String RECORD_READER_SERVICE = "record-reader-service"; - public static final String CATALOG_SERVICE = "catalog-service"; - public static final String CATALOG_NAME = "iceberg_test"; - - public static final Namespace NAMESPACE = Namespace.of(CATALOG_NAME); - - /** - * Validates whether the table contains the expected records. The results should be sorted by a unique key, so we do not end up with flaky tests. - * - * @param table The table we should read the records from - * @param expected The expected list of Records - * @param sortBy The column position by which we will sort - * @throws IOException Exceptions when reading the table data - */ - public static void validateData(Table table, List expected, int sortBy) throws IOException { - List records = Lists.newArrayListWithExpectedSize(expected.size()); - try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { - iterable.forEach(records::add); - } - - validateData(expected, records, sortBy); - } - - /** - * Validates whether the 2 sets of records are the same. The results should be sorted by a unique key, so we do not end up with flaky tests. - * - * @param expected The expected list of Records - * @param actual The actual list of Records - * @param sortBy The column position by which we will sort - */ - public static void validateData(List expected, List actual, int sortBy) { - List sortedExpected = Lists.newArrayList(expected); - List sortedActual = Lists.newArrayList(actual); - // Sort based on the specified field - sortedExpected.sort(Comparator.comparingInt(record -> record.get(sortBy).hashCode())); - sortedActual.sort(Comparator.comparingInt(record -> record.get(sortBy).hashCode())); - - assertEquals(sortedExpected.size(), sortedActual.size()); - for (int i = 0; i < expected.size(); ++i) { - assertEquals(sortedExpected.get(i), sortedActual.get(i)); - } - } - - /** - * Validates the number of files under a {@link Table} - * - * @param tableLocation The location of table we are checking - * @param numberOfDataFiles The expected number of data files (TABLE_LOCATION/data/*) - */ - public static void validateNumberOfDataFiles(String tableLocation, int numberOfDataFiles) throws IOException { - List dataFiles = Files.walk(Paths.get(tableLocation + "/data")) - .filter(Files::isRegularFile) - .filter(path -> !path.getFileName().toString().startsWith(".")) - .toList(); - - assertEquals(numberOfDataFiles, dataFiles.size()); - } - - public static void validatePartitionFolders(String tableLocation, List partitionPaths) { - for (String partitionPath : partitionPaths) { - Path path = Paths.get(tableLocation + "/data/" + partitionPath); - assertTrue(Files.exists(path), "The expected path doesn't exists: " + path); - } - } - - public static String getSystemTemporaryDirectory() { - return System.getProperty("java.io.tmpdir"); - } - - public static File createTemporaryDirectory() { - return Paths.get(getSystemTemporaryDirectory(), UUID.randomUUID().toString()).toFile(); - } - - public static class RecordsBuilder { - - private final List records = new ArrayList<>(); - private final Schema schema; - - private RecordsBuilder(Schema schema) { - this.schema = schema; - } - - public RecordsBuilder add(Object... values) { - Validate.isTrue(schema.columns().size() == values.length, "Number of provided values and schema length should be equal."); - - GenericRecord record = GenericRecord.create(schema); - - for (int i = 0; i < values.length; i++) { - record.set(i, values[i]); - } - - records.add(record); - return this; - } - - public List build() { - return Collections.unmodifiableList(records); - } - - public static RecordsBuilder newInstance(Schema schema) { - return new RecordsBuilder(schema); - } - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/date.avsc b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/date.avsc deleted file mode 100644 index 6fc6010b78..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/date.avsc +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -{ - "namespace": "nifi", - "name": "data_types", - "type": "record", - "fields": [ - { - "name" : "timeMicros", - "type": { - "type": "long", - "logicalType": "time-micros" - } - }, - { - "name" : "timestampMicros", - "type": { - "type" : "long", - "logicalType" : "timestamp-micros" - } - }, - { - "name" : "date", - "type": { - "type" : "int", - "logicalType" : "date" - } - } - ] -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/secured-core-site.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/secured-core-site.xml deleted file mode 100644 index 0fd06a5383..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/secured-core-site.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - - hadoop.security.authentication - kerberos - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/unsecured-core-site.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/unsecured-core-site.xml deleted file mode 100644 index d590a5039c..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/unsecured-core-site.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - - hadoop.security.authentication - simple - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/user.avsc b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/user.avsc deleted file mode 100644 index 799c0023b4..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/test/resources/user.avsc +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -{ - "namespace": "nifi", - "type": "record", - "name": "User", - "fields": [ - { - "name": "id", - "type": ["long", "null"] - }, - { - "name": "name", - "type": ["string", "null"] - }, - { - "name": "department", - "type": { - "name": "Department", - "type": "enum", - "symbols": ["Finance", "Marketing", "Sales"] - } - } - ] -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/pom.xml deleted file mode 100644 index 4e7d773945..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/pom.xml +++ /dev/null @@ -1,40 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-services-api-nar - nar - - - - org.apache.nifi - nifi-iceberg-services-api - 2.0.0-SNAPSHOT - - - org.apache.nifi - nifi-standard-services-api-nar - 2.0.0-SNAPSHOT - nar - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/LICENSE b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/LICENSE deleted file mode 100644 index 6effaa89d3..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/LICENSE +++ /dev/null @@ -1,209 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -APACHE NIFI SUBCOMPONENTS: - -The Apache NiFi project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/NOTICE b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/NOTICE deleted file mode 100644 index 1c764a0630..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api-nar/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,501 +0,0 @@ -nifi-iceberg-processors-nar -Copyright 2014-2024 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -=========================================== -Apache Software License v2 -=========================================== - - (ASLv2) Apache Iceberg - The following NOTICE information applies: - Apache Iceberg - Copyright 2017-2022 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - (ASLv2) Apache Ant - The following NOTICE information applies: - Apache Ant - Copyright 1999-2016 The Apache Software Foundation - - (ASLv2) Apache Commons Codec - The following NOTICE information applies: - Apache Commons Codec - Copyright 2002-2014 The Apache Software Foundation - - src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java - contains test data from http://aspell.net/test/orig/batch0.tab. - Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - - =============================================================================== - - The content of package org.apache.commons.codec.language.bm has been translated - from the original php source code available at http://stevemorse.org/phoneticinfo.htm - with permission from the original authors. - Original source copyright: - Copyright (c) 2008 Alexander Beider & Stephen P. Morse. - - (ASLv2) Apache Commons DBCP - The following NOTICE information applies: - Apache Commons DBCP - Copyright 2001-2015 The Apache Software Foundation. - - (ASLv2) Apache HttpComponents - The following NOTICE information applies: - Apache HttpComponents Client - Copyright 1999-2016 The Apache Software Foundation - Apache HttpComponents Core - HttpCore - Copyright 2006-2009 The Apache Software Foundation - - (ASLv2) Apache Commons Pool - The following NOTICE information applies: - Apache Commons Pool - Copyright 1999-2009 The Apache Software Foundation. - - (ASLv2) Apache Commons BeanUtils - The following NOTICE information applies: - Apache Commons BeanUtils - Copyright 2000-2016 The Apache Software Foundation - - (ASLv2) Apache Commons IO - The following NOTICE information applies: - Apache Commons IO - Copyright 2002-2016 The Apache Software Foundation - - (ASLv2) Apache Commons Net - The following NOTICE information applies: - Apache Commons Net - Copyright 2001-2016 The Apache Software Foundation - - (ASLv2) Apache Avro - The following NOTICE information applies: - Apache Avro - Copyright 2009-2017 The Apache Software Foundation - - (ASLv2) Apache Parquet - The following NOTICE information applies: - Apache Parquet MR (Incubating) - Copyright 2014 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - (ASLv2) Audience Annotations - The following NOTICE information applies: - Apache Yetus - Copyright 2008-2018 The Apache Software Foundation - - (ASLv2) Apache Commons Compress - The following NOTICE information applies: - Apache Commons Compress - Copyright 2002-2017 The Apache Software Foundation - - The files in the package org.apache.commons.compress.archivers.sevenz - were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), - which has been placed in the public domain: - - "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) - - (ASLv2) Apache Commons Configuration - The following NOTICE information applies: - Apache Commons Configuration - Copyright 2001-2017 The Apache Software Foundation - - (ASLv2) Apache Commons Text - The following NOTICE information applies: - Apache Commons Text - Copyright 2001-2018 The Apache Software Foundation - - (ASLv2) Apache Commons CLI - The following NOTICE information applies: - Apache Commons CLI - Copyright 2001-2017 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - (ASLv2) Apache Commons Collections - The following NOTICE information applies: - Apache Commons Collections - Copyright 2001-2016 The Apache Software Foundation - - (ASLv2) Apache Commons Daemon - The following NOTICE information applies: - Apache Commons Daemon - Copyright 2002-2013 The Apache Software Foundation - - (ASLv2) Apache Ivy - The following NOTICE information applies: - Copyright 2007-2017 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - Portions of Ivy were originally developed at - Jayasoft SARL (http://www.jayasoft.fr/) - and are licensed to the Apache Software Foundation under the - "Software Grant License Agreement" - - SSH and SFTP support is provided by the JCraft JSch package, - which is open source software, available under - the terms of a BSD style license. - The original software and related information is available - at http://www.jcraft.com/jsch/. - - (ASLv2) Apache Commons Math - The following NOTICE information applies: - Apache Commons Math - Copyright 2001-2012 The Apache Software Foundation - - (ASLv2) Apache Hive - The following NOTICE information applies: - Apache Hive - Copyright 2008-2015 The Apache Software Foundation - - This product includes software developed by The Apache Software - Foundation (http://www.apache.org/). - - This product includes Jersey (https://jersey.java.net/) - Copyright (c) 2010-2014 Oracle and/or its affiliates. - - This project includes software copyrighted by Microsoft Corporation and - licensed under the Apache License, Version 2.0. - - This project includes software copyrighted by Dell SecureWorks and - licensed under the Apache License, Version 2.0. - - (ASLv2) Jackson JSON processor - The following NOTICE information applies: - # Jackson JSON processor - - Jackson is a high-performance, Free/Open Source JSON processing library. - It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has - been in development since 2007. - It is currently developed by a community of developers, as well as supported - commercially by FasterXML.com. - - ## Licensing - - Jackson core and extension components may licensed under different licenses. - To find the details that apply to this artifact see the accompanying LICENSE file. - For more information, including possible other licensing options, contact - FasterXML.com (http://fasterxml.com). - - ## Credits - - A list of contributors may be found from CREDITS file, which is included - in some artifacts (usually source distributions); but is always available - from the source code management (SCM) system project uses. - - (ASLv2) BoneCP - The following NOTICE information applies: - BoneCP - Copyright 2010 Wallace Wadge - - (ASLv2) Apache Hadoop - The following NOTICE information applies: - The binary distribution of this product bundles binaries of - org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the - following notices: - * Copyright 2011 Dain Sundstrom - * Copyright 2011 FuseSource Corp. http://fusesource.com - - The binary distribution of this product bundles binaries of - org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni), - which has the following notices: - * This product includes software developed by FuseSource Corp. - http://fusesource.com - * This product includes software developed at - Progress Software Corporation and/or its subsidiaries or affiliates. - * This product includes software developed by IBM Corporation and others. - - (ASLv2) Apache Commons Lang - The following NOTICE information applies: - Apache Commons Lang - Copyright 2001-2015 The Apache Software Foundation - - (ASLv2) Apache Curator - The following NOTICE information applies: - Curator Framework - Copyright 2011-2014 The Apache Software Foundation - - Curator Client - Copyright 2011-2014 The Apache Software Foundation - - Curator Recipes - Copyright 2011-2014 The Apache Software Foundation - - (ASLv2) Apache Derby - The following NOTICE information applies: - Apache Derby - Copyright 2004-2014 Apache, Apache DB, Apache Derby, Apache Torque, Apache JDO, Apache DDLUtils, - the Derby hat logo, the Apache JDO logo, and the Apache feather logo are trademarks of The Apache Software Foundation. - - (ASLv2) Apache Geronimo - The following NOTICE information applies: - Apache Geronimo - Copyright 2003-2008 The Apache Software Foundation - - (ASLv2) Jettison - The following NOTICE information applies: - Copyright 2006 Envoi Solutions LLC - - (ASLv2) Jetty - The following NOTICE information applies: - Jetty Web Container - Copyright 1995-2019 Mort Bay Consulting Pty Ltd. - - (ASLv2) Apache log4j - The following NOTICE information applies: - Apache log4j - Copyright 2007 The Apache Software Foundation - - (ASLv2) Apache Thrift - The following NOTICE information applies: - Apache Thrift - Copyright 2006-2010 The Apache Software Foundation. - - (ASLv2) Dropwizard Metrics - The following NOTICE information applies: - Metrics - Copyright 2010-2013 Coda Hale and Yammer, Inc. - - This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, - LongAdder), which was released with the following comments: - - Written by Doug Lea with assistance from members of JCP JSR-166 - Expert Group and released to the public domain, as explained at - http://creativecommons.org/publicdomain/zero/1.0/ - - (ASLv2) Joda Time - The following NOTICE information applies: - This product includes software developed by - Joda.org (http://www.joda.org/). - - (ASLv2) The Netty Project - The following NOTICE information applies: - The Netty Project - Copyright 2011 The Netty Project - - (ASLv2) Apache ZooKeeper - The following NOTICE information applies: - Apache ZooKeeper - Copyright 2009-2012 The Apache Software Foundation - - (ASLv2) JPam - The following NOTICE information applies: - Copyright 2003-2006 Greg Luck - - (ASLv2) Groovy 2.4.16 (http://www.groovy-lang.org) - groovy-2.4.16-indy - groovy-json-2.4.16-indy - groovy-sql-2.4.16-indy - The following NOTICE information applies: - Apache Groovy - Copyright 2003-2018 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - This product includes/uses ANTLR (http://www.antlr2.org/) - developed by Terence Parr 1989-2006 - - (ASLv2) ASM Based Accessors Helper Used By JSON Smart (net.minidev:accessors-smart:jar:1.2 - http://www.minidev.net/) - The following NOTICE information applies: - ASM Based Accessors Helper Used By JSON Smart 1.2 - Copyright 2017, Uriel Chemouni - - (ASLv2) JSON Smart (net.minidev:json-smart:jar:2.3 - http://www.minidev.net/) - The following NOTICE information applies: - JSON Smart 2.3 - Copyright 2017, Uriel Chemouni, Eitan Raviv - - (ASLv2) Nimbus JOSE+JWT (com.nimbusds:nimbus-jose-jwt - https://connect2id.com/products/nimbus-jose-jwt) - The following NOTICE information applies: - Nimbus JOSE+JWT - Copyright 2021, Connect2id Ltd. - - (ASLv2) Woodstox (com.fasterxml.woodstox:woodstox-core:bundle:5.3.0 - https://github.com/FasterXML/woodstox) - The following NOTICE information applies: - Woodstox Core 5.3.0 - Copyright 2015, FasterXML, LLC - - (ASLv2) Joda Time - The following NOTICE information applies: - This product includes software developed by - Joda.org (http://www.joda.org/). - - (ASLv2) java-util - The following NOTICE information applies: - java-util - Copyright 2011-2017 Metamarkets Group Inc. - - (ASLv2) JCIP Annotations Under Apache License - The following NOTICE information applies: - JCIP Annotations Under Apache License - Copyright 2013 Stephen Connolly. - - (ASLv2) Google GSON - The following NOTICE information applies: - Copyright 2008 Google Inc. - - (ASLv2) Guava - The following NOTICE information applies: - Guava - Copyright 2015 The Guava Authors - - (ASLv2) OkHttp - The following NOTICE information applies: - OkHttp - Copyright (C) 2014 Square, Inc. - - (ASLv2) Okio - The following NOTICE information applies: - Okio - Copyright (C) 2014 Square, Inc. - - (ASLv2) Dropwizard Metrics - The following NOTICE information applies: - Dropwizard Metrics - Copyright 2010-2013 Coda Hale and Yammer, Inc., 2014-2017 Dropwizard Team - - (ASLv2) atinject (javax.inject:javax.inject) - The following NOTICE information applies: - atinject - Copyright - - (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3) - - (ASLv2) JetBrains/java-annotations - The following NOTICE information applies: - JetBrains/java-annotations - Copyright 2000-2016 JetBrains s.r.o. - - (ASLv2) Apache Kerby - The following NOTICE information applies: - Apache Kerby - Copyright 2003-2018 The Apache Software Foundation - - (ASLv2) Carrotsearch HPPC - The following NOTICE information applies: - HPPC borrowed code, ideas or both from: - - * Apache Lucene, http://lucene.apache.org/ - (Apache license) - * Fastutil, http://fastutil.di.unimi.it/ - (Apache license) - * Koloboke, https://github.com/OpenHFT/Koloboke - (Apache license) - - (ASLv2) Ehcache 2.x - The following NOTICE information applies: - Copyright 2003-2010 Terracotta, Inc. - - (ASLv2) Google Guice - The following NOTICE information applies: - Google Guice - Core Library - Copyright 2006-2011 Google, Inc. - - Google Guice - Extensions - Servlet - Copyright 2006-2011 Google, Inc. - - (ASLv2) Apache Arrow - The following NOTICE information applies: - Copyright 2016-2019 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). - - - (ASLv2) Apache ORC - The following NOTICE information applies: - Copyright 2013-2019 The Apache Software Foundation - - This product includes software developed by The Apache Software - Foundation (http://www.apache.org/). - - This product includes software developed by Hewlett-Packard: - (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P - -************************ -Common Development and Distribution License 1.1 -************************ - - The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details. - - (CDDL 1.1) (GPL2 w/ CPE) jersey-client (com.sun.jersey:jersey-client:jar:1.19 - https://jersey.java.net) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:jar:1.19 - https://jersey.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) Java Architecture For XML Binding (javax.xml.bind:jaxb-api:jar:2.2.11 - https://jaxb.dev.java.net/) - -************************ -Common Development and Distribution License 1.0 -************************ - - The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details. - - (CDDL 1.0) JavaServlet(TM) Specification (javax.servlet:servlet-api:jar:3.1.0 - no url available) - (CDDL 1.0) JavaServer Pages(TM) API (javax.servlet.jsp:jsp-api:jar:2.1 - http://jsp.java.net) - (CDDL 1.0) JSR311 API (javax.ws.rs:jsr311-api:jar:1.1.1 - https://jsr311.dev.java.net) - -***************** -Public Domain -***************** - - The following binary components are provided to the 'Public Domain'. See project link for details. - - (Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/) - - -************************ -Eclipse Distribution License 1.0 -************************ - - The following binary components are provided under the Eclipse Distribution License 1.0. - - (EDL 1.0) Jakarta Activation API (jakarta.activation:jakarta.activation-api:jar:1.2.1) - (EDL 1.0) Jakarta XML Binding API (jakarta.xml.bind:jakarta.xml.bind-api:jar:2.3.3) - -************************ -Eclipse Public License 2.0 -************************ - - The following binary components are provided under the Eclipse Public License 2.0. - - (EPL 2.0) javax.ws.rs-api (https://github.com/eclipse-ee4j/jaxrs-api) javax.ws.rs:javax.ws.rs-api:bundle:2.1.1 - -************************ -BSD License -************************ - - (BSD) JSch - The following NOTICE information applies: - Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. - All rights reserved. - https://www.jcraft.com/jsch/ - - (BSD 3-Clause) JLine Bundle - The following NOTICE information applies: - Copyright (c) 2002-2007, Marc Prud'hommeaux. All rights reserved. - https://github.com/jline/jline1 - - (BSD 3-Clause) ThreeTen-Extra - The following NOTICE information applies: - Copyright (c) 2007-2022, Stephen Colebourne & Michael Nascimento Santos. - https://github.com/ThreeTen/threeten-extra/ - -************************ -Go License -************************ - -The following binary components are provided under the Go License. See project link for details. - - (Go) RE2/J - The following NOTICE information applies: - Copyright (c) 2009 The Go Authors. All rights reserved. - https://github.com/google/re2j \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/pom.xml deleted file mode 100644 index f4b7fc9484..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/pom.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-services-api - jar - - - - - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogProperty.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogProperty.java deleted file mode 100644 index 0874f731ad..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogProperty.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -public enum IcebergCatalogProperty { - - CATALOG_NAME, - METASTORE_URI("hive.metastore.uris"), - WAREHOUSE_LOCATION("hive.metastore.warehouse.dir"), - CLIENT_POOL_SERVICE; - - private static final String EMPTY_STRING = ""; - - private final String hadoopPropertyName; - - IcebergCatalogProperty() { - this.hadoopPropertyName = EMPTY_STRING; - } - - IcebergCatalogProperty(String hadoopPropertyName) { - this.hadoopPropertyName = hadoopPropertyName; - } - - public String getHadoopPropertyName() { - return hadoopPropertyName; - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogService.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogService.java deleted file mode 100644 index 065d5290ef..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogService.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -import org.apache.nifi.controller.ControllerService; - -import java.util.List; -import java.util.Map; - -/** - * Provides a basic connector to Iceberg catalog services. - */ -public interface IcebergCatalogService extends ControllerService { - - IcebergCatalogType getCatalogType(); - - Map getCatalogProperties(); - - List getConfigFilePaths(); -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogType.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogType.java deleted file mode 100644 index 8aad41049e..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogType.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -public enum IcebergCatalogType { - HIVE, - HADOOP, - JDBC -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/pom.xml deleted file mode 100644 index 1edbf0d3c8..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/pom.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-services-nar - nar - - - - org.apache.nifi - nifi-iceberg-services - 2.0.0-SNAPSHOT - - - org.apache.nifi - nifi-iceberg-services-api-nar - 2.0.0-SNAPSHOT - nar - - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/LICENSE b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/LICENSE deleted file mode 100644 index 6effaa89d3..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/LICENSE +++ /dev/null @@ -1,209 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -APACHE NIFI SUBCOMPONENTS: - -The Apache NiFi project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/NOTICE b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/NOTICE deleted file mode 100644 index 9ca5225882..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services-nar/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,17 +0,0 @@ -nifi-iceberg-services-nar -Copyright 2014-2024 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -=========================================== -Apache Software License v2 -=========================================== - - (ASLv2) Apache Iceberg - The following NOTICE information applies: - Apache Iceberg - Copyright 2017-2022 The Apache Software Foundation - - This product includes software developed at - The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/pom.xml deleted file mode 100644 index dbc4a5419d..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/pom.xml +++ /dev/null @@ -1,50 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-services - jar - - - - org.apache.nifi - nifi-iceberg-services-api - 2.0.0-SNAPSHOT - provided - - - org.apache.nifi - nifi-dbcp-service-api - - - - org.apache.nifi - nifi-utils - - - org.apache.nifi - nifi-xml-processing - 2.0.0-SNAPSHOT - - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/AbstractCatalogService.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/AbstractCatalogService.java deleted file mode 100644 index d914348be9..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/AbstractCatalogService.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.resource.ResourceCardinality; -import org.apache.nifi.components.resource.ResourceType; -import org.apache.nifi.controller.AbstractControllerService; -import org.apache.nifi.expression.ExpressionLanguageScope; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.xml.processing.parsers.StandardDocumentProvider; -import org.w3c.dom.Document; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Abstract class holding common properties and methods for Catalog Service implementations. - */ -public abstract class AbstractCatalogService extends AbstractControllerService implements IcebergCatalogService { - - protected Map catalogProperties = new HashMap<>(); - - protected List configFilePaths; - - public static final PropertyDescriptor HADOOP_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder() - .name("hadoop-config-resources") - .displayName("Hadoop Configuration Resources") - .description("A file, or comma separated list of files, which contain the Hadoop configuration (core-site.xml, etc.). Without this, default configuration will be used.") - .required(false) - .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE) - .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT) - .build(); - - public static final PropertyDescriptor WAREHOUSE_PATH = new PropertyDescriptor.Builder() - .name("warehouse-path") - .displayName("Warehouse Path") - .description("Path to the location of the warehouse.") - .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT) - .required(true) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - protected List parseConfigFilePaths(String configFilePaths) { - List documentList = new ArrayList<>(); - for (final String configFile : createFilePathList(configFilePaths)) { - File file = new File(configFile.trim()); - try (final InputStream fis = new FileInputStream(file); - final InputStream in = new BufferedInputStream(fis)) { - final StandardDocumentProvider documentProvider = new StandardDocumentProvider(); - documentList.add(documentProvider.parse(in)); - } catch (IOException e) { - throw new ProcessException("Failed to load config files", e); - } - } - return documentList; - } - - protected List createFilePathList(String configFilePaths) { - List filePathList = new ArrayList<>(); - if (configFilePaths != null && !configFilePaths.trim().isEmpty()) { - for (final String configFile : configFilePaths.split(",")) { - filePathList.add(configFile.trim()); - } - } - return filePathList; - } - - @Override - public Map getCatalogProperties() { - return catalogProperties; - } - - @Override - public List getConfigFilePaths() { - return configFilePaths; - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HadoopCatalogService.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HadoopCatalogService.java deleted file mode 100644 index 42b364d23f..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HadoopCatalogService.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.controller.ConfigurationContext; - -import java.util.List; - -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.WAREHOUSE_LOCATION; - -@Tags({"iceberg", "catalog", "service", "hadoop", "hdfs"}) -@CapabilityDescription("Catalog service that can use HDFS or similar file systems that support atomic rename.") -public class HadoopCatalogService extends AbstractCatalogService { - - private static final List PROPERTIES = List.of(WAREHOUSE_PATH, HADOOP_CONFIGURATION_RESOURCES); - - @Override - protected List getSupportedPropertyDescriptors() { - return PROPERTIES; - } - - @OnEnabled - public void onEnabled(final ConfigurationContext context) { - if (context.getProperty(HADOOP_CONFIGURATION_RESOURCES).isSet()) { - configFilePaths = createFilePathList(context.getProperty(HADOOP_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue()); - } - - catalogProperties.put(WAREHOUSE_LOCATION, context.getProperty(WAREHOUSE_PATH).evaluateAttributeExpressions().getValue()); - } - - @Override - public IcebergCatalogType getCatalogType() { - return IcebergCatalogType.HADOOP; - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HiveCatalogService.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HiveCatalogService.java deleted file mode 100644 index c421c8c3b2..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/HiveCatalogService.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.ValidationContext; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.controller.ConfigurationContext; -import org.apache.nifi.expression.ExpressionLanguageScope; -import org.apache.nifi.processor.util.StandardValidators; -import org.w3c.dom.Document; -import org.w3c.dom.NodeList; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - -@Tags({"iceberg", "catalog", "service", "metastore", "hive"}) -@CapabilityDescription("Catalog service that connects to a Hive metastore to keep track of Iceberg tables.") -public class HiveCatalogService extends AbstractCatalogService { - - public static final PropertyDescriptor METASTORE_URI = new PropertyDescriptor.Builder() - .name("hive-metastore-uri") - .displayName("Hive Metastore URI") - .description("The URI location(s) for the Hive metastore; note that this is not the location of the Hive Server. The default port for the Hive metastore is 9043.") - .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT) - .addValidator(StandardValidators.URI_LIST_VALIDATOR) - .build(); - - public static final PropertyDescriptor WAREHOUSE_LOCATION = new PropertyDescriptor.Builder() - .name("warehouse-location") - .displayName("Default Warehouse Location") - .description("Location of default database for the warehouse. This field sets or overrides the 'hive.metastore.warehouse.dir' configuration property.") - .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT) - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .build(); - - private static final List PROPERTIES = List.of(METASTORE_URI, WAREHOUSE_LOCATION, HADOOP_CONFIGURATION_RESOURCES); - - @Override - protected List getSupportedPropertyDescriptors() { - return PROPERTIES; - } - - @Override - protected Collection customValidate(ValidationContext validationContext) { - - final List problems = new ArrayList<>(); - boolean configMetastoreUriPresent = false; - boolean configWarehouseLocationPresent = false; - - final String propertyMetastoreUri = validationContext.getProperty(METASTORE_URI).evaluateAttributeExpressions().getValue(); - final String propertyWarehouseLocation = validationContext.getProperty(WAREHOUSE_LOCATION).evaluateAttributeExpressions().getValue(); - - // Load the configurations for validation only if any config resource is provided and if either the metastore URI or the warehouse location property is missing - if (validationContext.getProperty(HADOOP_CONFIGURATION_RESOURCES).isSet() && (propertyMetastoreUri == null || propertyWarehouseLocation == null)) { - final String configFiles = validationContext.getProperty(HADOOP_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue(); - final List documents = parseConfigFilePaths(configFiles); - - for (Document document : documents) { - final NodeList nameNodeList = document.getElementsByTagName("name"); - - for (int i = 0; i < nameNodeList.getLength(); i++) { - final String nodeValue = nameNodeList.item(i).getFirstChild().getNodeValue(); - - if (nodeValue.equals(IcebergCatalogProperty.METASTORE_URI.getHadoopPropertyName())) { - configMetastoreUriPresent = true; - } - - if (nodeValue.equals(IcebergCatalogProperty.WAREHOUSE_LOCATION.getHadoopPropertyName())) { - configWarehouseLocationPresent = true; - } - - if (configMetastoreUriPresent && configWarehouseLocationPresent) { - break; - } - } - } - } - - if (!configMetastoreUriPresent && propertyMetastoreUri == null) { - problems.add(new ValidationResult.Builder() - .subject("Hive Metastore URI") - .valid(false) - .explanation("cannot find hive metastore uri, please provide it in the 'Hive Metastore URI' property" + - " or provide a configuration file which contains 'hive.metastore.uris' value.") - .build()); - } - - if (!configWarehouseLocationPresent && propertyWarehouseLocation == null) { - problems.add(new ValidationResult.Builder() - .subject("Default Warehouse Location") - .valid(false) - .explanation("cannot find default warehouse location, please provide it in the 'Default Warehouse Location' property" + - " or provide a configuration file which contains 'hive.metastore.warehouse.dir' value.") - .build()); - } - - return problems; - } - - @OnEnabled - public void onEnabled(final ConfigurationContext context) { - if (context.getProperty(METASTORE_URI).isSet()) { - catalogProperties.put(IcebergCatalogProperty.METASTORE_URI, context.getProperty(METASTORE_URI).evaluateAttributeExpressions().getValue()); - } - - if (context.getProperty(WAREHOUSE_LOCATION).isSet()) { - catalogProperties.put(IcebergCatalogProperty.WAREHOUSE_LOCATION, context.getProperty(WAREHOUSE_LOCATION).evaluateAttributeExpressions().getValue()); - } - - if (context.getProperty(HADOOP_CONFIGURATION_RESOURCES).isSet()) { - configFilePaths = createFilePathList(context.getProperty(HADOOP_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue()); - } - } - - @Override - public IcebergCatalogType getCatalogType() { - return IcebergCatalogType.HIVE; - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/JdbcCatalogService.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/JdbcCatalogService.java deleted file mode 100644 index 7f8be77e06..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/java/org/apache/nifi/services/iceberg/JdbcCatalogService.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.services.iceberg; - -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.controller.ConfigurationContext; -import org.apache.nifi.dbcp.DBCPService; -import org.apache.nifi.expression.ExpressionLanguageScope; -import org.apache.nifi.processor.util.StandardValidators; - -import java.util.List; - -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.CLIENT_POOL_SERVICE; -import static org.apache.nifi.services.iceberg.IcebergCatalogProperty.WAREHOUSE_LOCATION; - -@Tags({"iceberg", "catalog", "service", "jdbc"}) -@CapabilityDescription("Catalog service using relational database to manage Iceberg tables through JDBC.") -public class JdbcCatalogService extends AbstractCatalogService { - - public static final PropertyDescriptor CATALOG_NAME = new PropertyDescriptor.Builder() - .name("Catalog Name") - .description("Name of the Iceberg catalog.") - .addValidator(StandardValidators.NON_BLANK_VALIDATOR) - .defaultValue("jdbc-catalog") - .required(true) - .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT) - .build(); - - public static final PropertyDescriptor CONNECTION_POOL = new PropertyDescriptor.Builder() - .name("Connection Pool") - .description("Specifies the JDBC Connection Pool to use in order to communicate with the Iceberg catalog.") - .identifiesControllerService(DBCPService.class) - .required(true) - .build(); - - private static final List PROPERTIES = List.of( - CATALOG_NAME, CONNECTION_POOL, WAREHOUSE_PATH, HADOOP_CONFIGURATION_RESOURCES); - - @Override - protected List getSupportedPropertyDescriptors() { - return PROPERTIES; - } - - @OnEnabled - public void onEnabled(final ConfigurationContext context) { - if (context.getProperty(HADOOP_CONFIGURATION_RESOURCES).isSet()) { - configFilePaths = createFilePathList(context.getProperty(HADOOP_CONFIGURATION_RESOURCES).evaluateAttributeExpressions().getValue()); - } - - catalogProperties.put(IcebergCatalogProperty.CATALOG_NAME, context.getProperty(CATALOG_NAME).evaluateAttributeExpressions().getValue()); - catalogProperties.put(CLIENT_POOL_SERVICE, context.getProperty(CONNECTION_POOL).asControllerService(DBCPService.class)); - catalogProperties.put(WAREHOUSE_LOCATION, context.getProperty(WAREHOUSE_PATH).evaluateAttributeExpressions().getValue()); - } - - @Override - public IcebergCatalogType getCatalogType() { - return IcebergCatalogType.JDBC; - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService deleted file mode 100755 index 44a8e957cd..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ /dev/null @@ -1,18 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -org.apache.nifi.services.iceberg.HiveCatalogService -org.apache.nifi.services.iceberg.HadoopCatalogService -org.apache.nifi.services.iceberg.JdbcCatalogService \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/pom.xml deleted file mode 100644 index b3458a7a16..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/pom.xml +++ /dev/null @@ -1,173 +0,0 @@ - - - - - nifi-iceberg-bundle - org.apache.nifi - 2.0.0-SNAPSHOT - - 4.0.0 - - nifi-iceberg-test-utils - jar - - - - org.apache.hive - hive-metastore - ${hive.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - org.apache.logging.log4j - log4j-core - - - org.apache.logging.log4j - log4j-web - - - org.apache.logging.log4j - log4j-1.2-api - - - org.apache.logging.log4j - log4j-slf4j-impl - - - org.apache.orc - orc-core - - - org.apache.hbase - hbase-client - - - co.cask.tephra - tephra-api - - - co.cask.tephra - tephra-core - - - co.cask.tephra - tephra-hbase-compat-1.0 - - - org.apache.parquet - parquet-hadoop-bundle - - - com.tdunning - json - - - com.zaxxer - HikariCP - - - com.google.guava - guava - - - org.apache.groovy - groovy-all - - - org.apache.ivy - ivy - - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-webapp - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-http - - - org.eclipse.jetty - jetty-rewrite - - - org.apache.hive - hive-shims - - - com.jolbox - bonecp - - - commons-cli - commons-cli - - - com.google.protobuf - protobuf-java - - - - - org.apache.hadoop - hadoop-common - - - - org.apache.derby - derbytools - ${derby.version} - - - org.apache.derby - derby - ${derby.version} - - - org.junit.jupiter - junit-jupiter-api - compile - - - - \ No newline at end of file diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/MetastoreCore.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/MetastoreCore.java deleted file mode 100644 index 3d2908b3d1..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/MetastoreCore.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.hive.metastore; - -import org.apache.commons.io.FileUtils; -import org.apache.derby.jdbc.EmbeddedDriver; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStore; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IHMSHandler; -import org.apache.hadoop.hive.metastore.RetryingHMSHandler; -import org.apache.hadoop.hive.metastore.TServerSocketKeepAlive; -import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.thrift.TException; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.server.TServer; -import org.apache.thrift.server.TThreadPoolServer; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TTransportException; -import org.apache.thrift.transport.TTransportFactory; - -import java.io.BufferedReader; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.Reader; -import java.lang.reflect.InvocationTargetException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import static java.nio.file.Files.createTempDirectory; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.AUTO_CREATE_ALL; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.COMPACTOR_WORKER_THREADS; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.CONNECTION_DRIVER; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.CONNECT_URL_KEY; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_SUPPORT_CONCURRENCY; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_TXN_MANAGER; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HMS_HANDLER_FORCE_RELOAD_CONF; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.SCHEMA_VERIFICATION; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.THRIFT_URIS; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.WAREHOUSE; - -/** - * This class wraps Metastore service core functionalities. - */ -class MetastoreCore { - - private final String DATABASE_NAME = "iceberg_test"; - - private String thriftConnectionUri; - private Configuration hiveConf; - private HiveMetaStoreClient metaStoreClient; - private File tempDir; - private ExecutorService thriftServer; - private TServer server; - - public void initialize(Map configOverrides) throws IOException, TException, InvocationTargetException, NoSuchMethodException, - IllegalAccessException, NoSuchFieldException, SQLException { - thriftServer = Executors.newSingleThreadExecutor(); - tempDir = createTempDirectory("metastore").toFile(); - setDerbyLogPath(); - setupDB("jdbc:derby:" + getDerbyPath() + ";create=true"); - - server = thriftServer(configOverrides); - thriftServer.submit(() -> server.serve()); - - metaStoreClient = new HiveMetaStoreClient(hiveConf); - metaStoreClient.createDatabase(new Database(DATABASE_NAME, "description", getDBPath(), new HashMap<>())); - } - - public void shutdown() { - metaStoreClient.close(); - - if (server != null) { - server.stop(); - } - - thriftServer.shutdown(); - - FileUtils.deleteQuietly(tempDir); - } - - private HiveConf hiveConf(int port, Map configOverrides) throws IOException { - thriftConnectionUri = "thrift://localhost:" + port; - - final HiveConf hiveConf = new HiveConf(new Configuration(), this.getClass()); - hiveConf.set(THRIFT_URIS.getVarname(), thriftConnectionUri); - hiveConf.set(WAREHOUSE.getVarname(), "file:" + tempDir.getAbsolutePath()); - hiveConf.set(WAREHOUSE.getHiveName(), "file:" + tempDir.getAbsolutePath()); - hiveConf.set(CONNECTION_DRIVER.getVarname(), EmbeddedDriver.class.getName()); - hiveConf.set(CONNECT_URL_KEY.getVarname(), "jdbc:derby:" + getDerbyPath() + ";create=true"); - hiveConf.set(AUTO_CREATE_ALL.getVarname(), "false"); - hiveConf.set(SCHEMA_VERIFICATION.getVarname(), "false"); - hiveConf.set(HIVE_TXN_MANAGER.getVarname(), "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager"); - hiveConf.set(COMPACTOR_INITIATOR_ON.getVarname(), "true"); - hiveConf.set(COMPACTOR_WORKER_THREADS.getVarname(), "1"); - hiveConf.set(HIVE_SUPPORT_CONCURRENCY.getVarname(), "true"); - hiveConf.setBoolean("hcatalog.hive.client.cache.disabled", true); - - hiveConf.set(CONNECTION_POOLING_TYPE.getVarname(), "NONE"); - hiveConf.set(HMS_HANDLER_FORCE_RELOAD_CONF.getVarname(), "true"); - - configOverrides.forEach(hiveConf::set); - - writeHiveConfFile(hiveConf); - return hiveConf; - } - - private void setDerbyLogPath() throws IOException { - final String derbyLog = Files.createTempFile(tempDir.toPath(), "derby", ".log").toString(); - System.setProperty("derby.stream.error.file", derbyLog); - } - - private String getDerbyPath() { - return new File(tempDir, "metastore_db").getPath(); - } - - private TServer thriftServer(Map configOverrides) throws TTransportException, MetaException, InvocationTargetException, - NoSuchMethodException, IllegalAccessException, NoSuchFieldException, IOException { - final TServerSocketKeepAlive socket = new TServerSocketKeepAlive(new TServerSocket(0)); - hiveConf = hiveConf(socket.getServerSocket().getLocalPort(), configOverrides); - final HiveMetaStore.HMSHandler baseHandler = new HiveMetaStore.HMSHandler("new db based metaserver", hiveConf); - final IHMSHandler handler = RetryingHMSHandler.getProxy(hiveConf, baseHandler, true); - final TTransportFactory transportFactory = new TTransportFactory(); - final TSetIpAddressProcessor processor = new TSetIpAddressProcessor<>(handler); - - TThreadPoolServer.Args args = new TThreadPoolServer.Args(socket) - .processor(processor) - .transportFactory(transportFactory) - .protocolFactory(new TBinaryProtocol.Factory()) - .minWorkerThreads(3) - .maxWorkerThreads(5); - - return new TThreadPoolServer(args); - } - - private void setupDB(String dbURL) throws SQLException, IOException { - final Connection connection = DriverManager.getConnection(dbURL); - final ScriptRunner scriptRunner = new ScriptRunner(connection); - - final InputStream inputStream = getClass().getClassLoader().getResourceAsStream("hive-schema-4.0.0-alpha-2.derby.sql"); - final Reader reader = new BufferedReader(new InputStreamReader(inputStream)); - scriptRunner.runScript(reader); - } - - private String getDBPath() { - return Paths.get(tempDir.getAbsolutePath(), DATABASE_NAME + ".db").toAbsolutePath().toString(); - } - - private void writeHiveConfFile(HiveConf hiveConf) throws IOException { - File file = new File(tempDir.toPath() + "/hive-site.xml"); - hiveConf.writeXml(Files.newOutputStream(file.toPath())); - } - - public String getThriftConnectionUri() { - return thriftConnectionUri; - } - - public String getWarehouseLocation() { - return tempDir.getAbsolutePath(); - } - - public HiveMetaStoreClient getMetaStoreClient() { - return metaStoreClient; - } - - public Configuration getConfiguration() { - return hiveConf; - } - - public String getConfigurationLocation() { - return tempDir.toPath() + "/hive-site.xml"; - } - -} - diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ScriptRunner.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ScriptRunner.java deleted file mode 100644 index d3666fdbe4..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ScriptRunner.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.hive.metastore; - -import java.io.IOException; -import java.io.LineNumberReader; -import java.io.Reader; -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.Statement; - -/** This class is responsible for metastore init script processing and execution. */ -public class ScriptRunner { - - private static final String DEFAULT_DELIMITER = ";"; - - private final Connection connection; - - public ScriptRunner(Connection connection) throws SQLException { - this.connection = connection; - if (!this.connection.getAutoCommit()) { - // May throw SQLFeatureNotSupportedException which is a subclass of SQLException - this.connection.setAutoCommit(true); - } - } - - public void runScript(Reader reader) throws IOException, SQLException { - try { - StringBuilder command = new StringBuilder(); - LineNumberReader lineReader = new LineNumberReader(reader); - String line; - while ((line = lineReader.readLine()) != null) { - String trimmedLine = line.trim(); - if (trimmedLine.isEmpty() || trimmedLine.startsWith("--") || trimmedLine.startsWith("//")) { - continue; //Skip comment line - } else if (trimmedLine.endsWith(getDelimiter())) { - command.append(line, 0, line.lastIndexOf(getDelimiter())); - command.append(" "); - Statement statement = connection.createStatement(); - - statement.execute(command.toString()); - connection.commit(); - - command = new StringBuilder(); - - statement.close(); - } else { - command.append(line); - command.append(" "); - } - } - } catch (IOException | SQLException e) { - throw e; - } catch (Exception e) { - throw new RuntimeException("Error running metastore init script.", e); - } finally { - connection.rollback(); - } - } - - private String getDelimiter() { - return DEFAULT_DELIMITER; - } -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ThriftMetastore.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ThriftMetastore.java deleted file mode 100644 index 2b3dc4da5c..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/java/org/apache/nifi/hive/metastore/ThriftMetastore.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.hive.metastore; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.junit.jupiter.api.extension.AfterAllCallback; -import org.junit.jupiter.api.extension.BeforeAllCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -import java.util.HashMap; -import java.util.Map; - -/** A JUnit Extension that creates a Hive Metastore Thrift service backed by a Hive Metastore using an in-memory Derby database. */ -public class ThriftMetastore implements BeforeAllCallback, AfterAllCallback { - - private final MetastoreCore metastoreCore; - - private Map configOverrides = new HashMap<>(); - - public ThriftMetastore() { - metastoreCore = new MetastoreCore(); - } - - public ThriftMetastore withConfigOverrides(Map configs) { - configOverrides = configs; - return this; - } - - @Override - public void beforeAll(ExtensionContext context) throws Exception { - metastoreCore.initialize(configOverrides); - } - - @Override - public void afterAll(ExtensionContext context) { - metastoreCore.shutdown(); - } - - public String getThriftConnectionUri() { - return metastoreCore.getThriftConnectionUri(); - } - - public String getWarehouseLocation() { - return metastoreCore.getWarehouseLocation(); - } - - public HiveMetaStoreClient getMetaStoreClient() { - return metastoreCore.getMetaStoreClient(); - } - - public Configuration getConfiguration() { - return metastoreCore.getConfiguration(); - } - - public String getConfigurationLocation() { - return metastoreCore.getConfigurationLocation(); - } - -} diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/resources/hive-schema-4.0.0-alpha-2.derby.sql b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/resources/hive-schema-4.0.0-alpha-2.derby.sql deleted file mode 100644 index c1cc235a6f..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-test-utils/src/main/resources/hive-schema-4.0.0-alpha-2.derby.sql +++ /dev/null @@ -1,791 +0,0 @@ --- Licensed to the Apache Software Foundation (ASF) under one or more --- contributor license agreements. See the NOTICE file distributed with --- this work for additional information regarding copyright ownership. --- The ASF licenses this file to You under the Apache License, Version 2.0 --- (the "License"); you may not use this file except in compliance with --- the License. You may obtain a copy of the License at --- http://www.apache.org/licenses/LICENSE-2.0 --- Unless required by applicable law or agreed to in writing, software --- distributed under the License is distributed on an "AS IS" BASIS, --- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. --- See the License for the specific language governing permissions and --- limitations under the License. - --- Timestamp: 2011-09-22 15:32:02.024 --- Source database is: /home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb --- Connection URL is: jdbc:derby:/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb --- Specified schema is: APP --- appendLogs: false - --- ---------------------------------------------- --- DDL Statements for functions --- ---------------------------------------------- - -CREATE FUNCTION "APP"."NUCLEUS_ASCII" (C CHAR(1)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.ascii' ; - -CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.matches' ; - --- ---------------------------------------------- --- DDL Statements for tables --- ---------------------------------------------- -CREATE TABLE "APP"."DBS" ( - "DB_ID" BIGINT NOT NULL, - "DESC" VARCHAR(4000), - "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, - "NAME" VARCHAR(128), - "OWNER_NAME" VARCHAR(128), - "OWNER_TYPE" VARCHAR(10), - "CTLG_NAME" VARCHAR(256) NOT NULL DEFAULT 'hive', - "CREATE_TIME" INTEGER, - "DB_MANAGED_LOCATION_URI" VARCHAR(4000), - "TYPE" VARCHAR(32) DEFAULT 'NATIVE' NOT NULL, - "DATACONNECTOR_NAME" VARCHAR(128), - "REMOTE_DBNAME" VARCHAR(128) -); - -CREATE TABLE "APP"."TBL_PRIVS" ("TBL_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."DATABASE_PARAMS" ("DB_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); - -CREATE TABLE "APP"."TBL_COL_PRIVS" ("TBL_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_COL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(4000), "COLUMN_NAME" VARCHAR(767) NOT NULL, "TYPE_NAME" CLOB, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL); - -CREATE TABLE "APP"."PARTITION_KEY_VALS" ("PART_ID" BIGINT NOT NULL, "PART_KEY_VAL" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."DB_PRIVS" ("DB_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "DB_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."DC_PRIVS" ("DC_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "NAME" VARCHAR(128), "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "DC_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."IDXS" ("INDEX_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DEFERRED_REBUILD" CHAR(1) NOT NULL, "INDEX_HANDLER_CLASS" VARCHAR(4000), "INDEX_NAME" VARCHAR(128), "INDEX_TBL_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "ORIG_TBL_ID" BIGINT, "SD_ID" BIGINT); - -CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); - -CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT, "WRITE_ID" BIGINT DEFAULT 0); - -CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER); - -CREATE TABLE "APP"."PART_PRIVS" ("PART_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."ROLE_MAP" ("ROLE_GRANT_ID" BIGINT NOT NULL, "ADD_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "ROLE_ID" BIGINT); - -CREATE TABLE "APP"."TYPES" ("TYPES_ID" BIGINT NOT NULL, "TYPE_NAME" VARCHAR(128), "TYPE1" VARCHAR(767), "TYPE2" VARCHAR(767)); - -CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "USER_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."PARTITION_PARAMS" ("PART_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."PARTITION_EVENTS" ( - "PART_NAME_ID" BIGINT NOT NULL, - "CAT_NAME" VARCHAR(256), - "DB_NAME" VARCHAR(128), - "EVENT_TIME" BIGINT NOT NULL, - "EVENT_TYPE" INTEGER NOT NULL, - "PARTITION_NAME" VARCHAR(767), - "TBL_NAME" VARCHAR(256) -); - -CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "COLUMN_NAME" VARCHAR(128) NOT NULL, "TYPE_NAME" VARCHAR(4000) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128)); - -CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "OWNER_TYPE" VARCHAR(10), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(256), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL DEFAULT 'N', "WRITE_ID" BIGINT DEFAULT 0); - -CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."PART_COL_PRIVS" ("PART_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_COL_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); - -CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000), "IS_COMPRESSED" CHAR(1) NOT NULL, "LOCATION" VARCHAR(4000), "NUM_BUCKETS" INTEGER NOT NULL, "OUTPUT_FORMAT" VARCHAR(4000), "SERDE_ID" BIGINT, "CD_ID" BIGINT, "IS_STOREDASSUBDIRECTORIES" CHAR(1) NOT NULL); - -CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL); - -CREATE TABLE "APP"."TAB_COL_STATS"( - "CAT_NAME" VARCHAR(256) NOT NULL, - "DB_NAME" VARCHAR(128) NOT NULL, - "TABLE_NAME" VARCHAR(256) NOT NULL, - "COLUMN_NAME" VARCHAR(767) NOT NULL, - "COLUMN_TYPE" VARCHAR(128) NOT NULL, - "LONG_LOW_VALUE" BIGINT, - "LONG_HIGH_VALUE" BIGINT, - "DOUBLE_LOW_VALUE" DOUBLE, - "DOUBLE_HIGH_VALUE" DOUBLE, - "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), - "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000), - "NUM_DISTINCTS" BIGINT, - "NUM_NULLS" BIGINT NOT NULL, - "AVG_COL_LEN" DOUBLE, - "MAX_COL_LEN" BIGINT, - "NUM_TRUES" BIGINT, - "NUM_FALSES" BIGINT, - "LAST_ANALYZED" BIGINT, - "CS_ID" BIGINT NOT NULL, - "TBL_ID" BIGINT NOT NULL, - "BIT_VECTOR" BLOB, - "ENGINE" VARCHAR(128) NOT NULL -); - -CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."BUCKETING_COLS" ("SD_ID" BIGINT NOT NULL, "BUCKET_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."TYPE_FIELDS" ("TYPE_NAME" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "FIELD_NAME" VARCHAR(128) NOT NULL, "FIELD_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."NUCLEUS_TABLES" ("CLASS_NAME" VARCHAR(128) NOT NULL, "TABLE_NAME" VARCHAR(128) NOT NULL, "TYPE" VARCHAR(4) NOT NULL, "OWNER" VARCHAR(2) NOT NULL, "VERSION" VARCHAR(20) NOT NULL, "INTERFACE_NAME" VARCHAR(256) DEFAULT NULL); - -CREATE TABLE "APP"."SD_PARAMS" ("SD_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); - -CREATE TABLE "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID" BIGINT NOT NULL); - -CREATE TABLE "APP"."SKEWED_STRING_LIST_VALUES" ("STRING_LIST_ID" BIGINT NOT NULL, "STRING_LIST_VALUE" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SKEWED_COL_NAMES" ("SD_ID" BIGINT NOT NULL, "SKEWED_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ("SD_ID" BIGINT NOT NULL, "STRING_LIST_ID_KID" BIGINT NOT NULL, "LOCATION" VARCHAR(4000)); - -CREATE TABLE "APP"."SKEWED_VALUES" ("SD_ID_OID" BIGINT NOT NULL, "STRING_LIST_ID_EID" BIGINT NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."MASTER_KEYS" ("KEY_ID" INTEGER NOT NULL generated always as identity (start with 1), "MASTER_KEY" VARCHAR(767)); - -CREATE TABLE "APP"."DELEGATION_TOKENS" ( "TOKEN_IDENT" VARCHAR(767) NOT NULL, "TOKEN" VARCHAR(767)); - -CREATE TABLE "APP"."PART_COL_STATS"( - "CAT_NAME" VARCHAR(256) NOT NULL, - "DB_NAME" VARCHAR(128) NOT NULL, - "TABLE_NAME" VARCHAR(256) NOT NULL, - "PARTITION_NAME" VARCHAR(767) NOT NULL, - "COLUMN_NAME" VARCHAR(767) NOT NULL, - "COLUMN_TYPE" VARCHAR(128) NOT NULL, - "LONG_LOW_VALUE" BIGINT, - "LONG_HIGH_VALUE" BIGINT, - "DOUBLE_LOW_VALUE" DOUBLE, - "DOUBLE_HIGH_VALUE" DOUBLE, - "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), - "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000), - "NUM_DISTINCTS" BIGINT, - "BIT_VECTOR" BLOB, - "NUM_NULLS" BIGINT NOT NULL, - "AVG_COL_LEN" DOUBLE, - "MAX_COL_LEN" BIGINT, - "NUM_TRUES" BIGINT, - "NUM_FALSES" BIGINT, - "LAST_ANALYZED" BIGINT, - "CS_ID" BIGINT NOT NULL, - "PART_ID" BIGINT NOT NULL, - "ENGINE" VARCHAR(128) NOT NULL -); - -CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255)); - -CREATE TABLE "APP"."FUNCS" ("FUNC_ID" BIGINT NOT NULL, "CLASS_NAME" VARCHAR(4000), "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "FUNC_NAME" VARCHAR(128), "FUNC_TYPE" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "OWNER_TYPE" VARCHAR(10)); - -CREATE TABLE "APP"."FUNC_RU" ("FUNC_ID" BIGINT NOT NULL, "RESOURCE_TYPE" INTEGER NOT NULL, "RESOURCE_URI" VARCHAR(4000), "INTEGER_IDX" INTEGER NOT NULL); - -CREATE TABLE "APP"."NOTIFICATION_LOG" ( - "NL_ID" BIGINT NOT NULL, - "CAT_NAME" VARCHAR(256), - "DB_NAME" VARCHAR(128), - "EVENT_ID" BIGINT NOT NULL, - "EVENT_TIME" INTEGER NOT NULL, - "EVENT_TYPE" VARCHAR(32) NOT NULL, - "MESSAGE" CLOB, - "TBL_NAME" VARCHAR(256), - "MESSAGE_FORMAT" VARCHAR(16) -); - -CREATE UNIQUE INDEX "APP"."NOTIFICATION_LOG_EVENT_ID" ON "APP"."NOTIFICATION_LOG" ("EVENT_ID"); - -CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL); - -CREATE TABLE "APP"."KEY_CONSTRAINTS" ( - "CHILD_CD_ID" BIGINT, - "CHILD_INTEGER_IDX" INTEGER, - "CHILD_TBL_ID" BIGINT, - "PARENT_CD_ID" BIGINT, - "PARENT_INTEGER_IDX" INTEGER, - "PARENT_TBL_ID" BIGINT NOT NULL, - "POSITION" BIGINT NOT NULL, - "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, - "CONSTRAINT_TYPE" SMALLINT NOT NULL, - "UPDATE_RULE" SMALLINT, - "DELETE_RULE" SMALLINT, - "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL, - "DEFAULT_VALUE" VARCHAR(400) -); - -CREATE TABLE "APP"."METASTORE_DB_PROPERTIES" ("PROPERTY_KEY" VARCHAR(255) NOT NULL, "PROPERTY_VALUE" VARCHAR(1000) NOT NULL, "DESCRIPTION" VARCHAR(1000)); - -CREATE TABLE "APP"."WM_RESOURCEPLAN" (RP_ID BIGINT NOT NULL, NS VARCHAR(128), NAME VARCHAR(128) NOT NULL, QUERY_PARALLELISM INTEGER, STATUS VARCHAR(20) NOT NULL, DEFAULT_POOL_ID BIGINT); - -CREATE TABLE "APP"."WM_POOL" (POOL_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, PATH VARCHAR(1024) NOT NULL, ALLOC_FRACTION DOUBLE, QUERY_PARALLELISM INTEGER, SCHEDULING_POLICY VARCHAR(1024)); - -CREATE TABLE "APP"."WM_TRIGGER" (TRIGGER_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, NAME VARCHAR(128) NOT NULL, TRIGGER_EXPRESSION VARCHAR(1024), ACTION_EXPRESSION VARCHAR(1024), IS_IN_UNMANAGED INTEGER NOT NULL DEFAULT 0); - -CREATE TABLE "APP"."WM_POOL_TO_TRIGGER" (POOL_ID BIGINT NOT NULL, TRIGGER_ID BIGINT NOT NULL); - -CREATE TABLE "APP"."WM_MAPPING" (MAPPING_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, ENTITY_TYPE VARCHAR(128) NOT NULL, ENTITY_NAME VARCHAR(128) NOT NULL, POOL_ID BIGINT, ORDERING INTEGER); - -CREATE TABLE "APP"."MV_CREATION_METADATA" ( - "MV_CREATION_METADATA_ID" BIGINT NOT NULL, - "CAT_NAME" VARCHAR(256) NOT NULL, - "DB_NAME" VARCHAR(128) NOT NULL, - "TBL_NAME" VARCHAR(256) NOT NULL, - "TXN_LIST" CLOB, - "MATERIALIZATION_TIME" BIGINT NOT NULL -); - -CREATE TABLE "APP"."MV_TABLES_USED" ( - "MV_CREATION_METADATA_ID" BIGINT NOT NULL, - "TBL_ID" BIGINT NOT NULL, - "INSERTED_COUNT" BIGINT NOT NULL DEFAULT 0, - "UPDATED_COUNT" BIGINT NOT NULL DEFAULT 0, - "DELETED_COUNT" BIGINT NOT NULL DEFAULT 0 -); - -CREATE TABLE "APP"."CTLGS" ( - "CTLG_ID" BIGINT NOT NULL, - "NAME" VARCHAR(256) UNIQUE, - "DESC" VARCHAR(4000), - "LOCATION_URI" VARCHAR(4000) NOT NULL, - "CREATE_TIME" INTEGER); - --- Insert a default value. The location is TBD. Hive will fix this when it starts -INSERT INTO "APP"."CTLGS" ("CTLG_ID", "NAME", "DESC", "LOCATION_URI", "CREATE_TIME") -VALUES (1, 'hive', 'Default catalog for Hive', 'TBD', NULL); - --- ---------------------------------------------- --- DML Statements --- ---------------------------------------------- - -INSERT INTO "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID", "NEXT_EVENT_ID") SELECT * FROM (VALUES (1,1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_EVENT_ID" FROM "APP"."NOTIFICATION_SEQUENCE"); - -INSERT INTO "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_VAL" FROM "APP"."SEQUENCE_TABLE" WHERE "SEQUENCE_NAME" = 'org.apache.hadoop.hive.metastore.model.MNotificationLog'); - --- ---------------------------------------------- --- DDL Statements for indexes --- ---------------------------------------------- - -CREATE UNIQUE INDEX "APP"."UNIQUEINDEX" ON "APP"."IDXS" ("INDEX_NAME", "ORIG_TBL_ID"); - -CREATE INDEX "APP"."TABLECOLUMNPRIVILEGEINDEX" ON "APP"."TBL_COL_PRIVS" ("AUTHORIZER", "TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."DBPRIVILEGEINDEX" ON "APP"."DB_PRIVS" ("AUTHORIZER", "DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."DCPRIVILEGEINDEX" ON "APP"."DC_PRIVS" ("AUTHORIZER", "NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DC_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME"); - -CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_NAME"); - -CREATE INDEX "APP"."PARTPRIVILEGEINDEX" ON "APP"."PART_PRIVS" ("AUTHORIZER", "PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."ROLEENTITYINDEX" ON "APP"."ROLES" ("ROLE_NAME"); - -CREATE INDEX "APP"."TABLEPRIVILEGEINDEX" ON "APP"."TBL_PRIVS" ("AUTHORIZER", "TBL_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."UNIQUETABLE" ON "APP"."TBLS" ("TBL_NAME", "DB_ID"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME"); - -CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."GLOBALPRIVILEGEINDEX" ON "APP"."GLOBAL_PRIVS" ("AUTHORIZER", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_TYPE" ON "APP"."TYPES" ("TYPE_NAME"); - -CREATE INDEX "APP"."PARTITIONCOLUMNPRIVILEGEINDEX" ON "APP"."PART_COL_PRIVS" ("AUTHORIZER", "PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); - -CREATE UNIQUE INDEX "APP"."UNIQUEPARTITION" ON "APP"."PARTITIONS" ("PART_NAME", "TBL_ID"); - -CREATE UNIQUE INDEX "APP"."UNIQUEFUNCTION" ON "APP"."FUNCS" ("FUNC_NAME", "DB_ID"); - -CREATE INDEX "APP"."FUNCS_N49" ON "APP"."FUNCS" ("DB_ID"); - -CREATE INDEX "APP"."FUNC_RU_N49" ON "APP"."FUNC_RU" ("FUNC_ID"); - -CREATE INDEX "APP"."CONSTRAINTS_PARENT_TBL_ID_INDEX" ON "APP"."KEY_CONSTRAINTS"("PARENT_TBL_ID"); - -CREATE INDEX "APP"."CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "APP"."KEY_CONSTRAINTS"("CONSTRAINT_TYPE"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_WM_RESOURCEPLAN" ON "APP"."WM_RESOURCEPLAN" ("NS", "NAME"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_WM_POOL" ON "APP"."WM_POOL" ("RP_ID", "PATH"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_WM_TRIGGER" ON "APP"."WM_TRIGGER" ("RP_ID", "NAME"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_WM_MAPPING" ON "APP"."WM_MAPPING" ("RP_ID", "ENTITY_TYPE", "ENTITY_NAME"); - -CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME"); - -CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); - - --- ---------------------------------------------- --- DDL Statements for keys --- ---------------------------------------------- - --- primary/unique -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_PK" PRIMARY KEY ("INDEX_ID"); - -ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_PK" PRIMARY KEY ("TBL_COLUMN_GRANT_ID"); - -ALTER TABLE "APP"."CDS" ADD CONSTRAINT "SQL110922153006460" PRIMARY KEY ("CD_ID"); - -ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_PK" PRIMARY KEY ("DB_GRANT_ID"); - -ALTER TABLE "APP"."DC_PRIVS" ADD CONSTRAINT "DC_PRIVS_PK" PRIMARY KEY ("DC_GRANT_ID"); - -ALTER TABLE "APP"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_PK" PRIMARY KEY ("INDEX_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEY_PK" PRIMARY KEY ("TBL_ID", "PKEY_NAME"); - -ALTER TABLE "APP"."SEQUENCE_TABLE" ADD CONSTRAINT "SEQUENCE_TABLE_PK" PRIMARY KEY ("SEQUENCE_NAME"); - -ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_PK" PRIMARY KEY ("PART_GRANT_ID"); - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_PK" PRIMARY KEY ("SD_ID"); - -ALTER TABLE "APP"."SERDES" ADD CONSTRAINT "SERDES_PK" PRIMARY KEY ("SERDE_ID"); - -ALTER TABLE "APP"."COLUMNS" ADD CONSTRAINT "COLUMNS_PK" PRIMARY KEY ("SD_ID", "COLUMN_NAME"); - -ALTER TABLE "APP"."PARTITION_EVENTS" ADD CONSTRAINT "PARTITION_EVENTS_PK" PRIMARY KEY ("PART_NAME_ID"); - -ALTER TABLE "APP"."TYPE_FIELDS" ADD CONSTRAINT "TYPE_FIELDS_PK" PRIMARY KEY ("TYPE_NAME", "FIELD_NAME"); - -ALTER TABLE "APP"."ROLES" ADD CONSTRAINT "ROLES_PK" PRIMARY KEY ("ROLE_ID"); - -ALTER TABLE "APP"."TBL_PRIVS" ADD CONSTRAINT "TBL_PRIVS_PK" PRIMARY KEY ("TBL_GRANT_ID"); - -ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_PK" PRIMARY KEY ("SERDE_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."NUCLEUS_TABLES" ADD CONSTRAINT "NUCLEUS_TABLES_PK" PRIMARY KEY ("CLASS_NAME"); - -ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_PK" PRIMARY KEY ("TBL_ID"); - -ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_PK" PRIMARY KEY ("SD_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_PK" PRIMARY KEY ("DB_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_PK" PRIMARY KEY ("DB_ID"); - -ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_PK" PRIMARY KEY ("ROLE_GRANT_ID"); - -ALTER TABLE "APP"."GLOBAL_PRIVS" ADD CONSTRAINT "GLOBAL_PRIVS_PK" PRIMARY KEY ("USER_GRANT_ID"); - -ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."PARTITION_KEY_VALS" ADD CONSTRAINT "PARTITION_KEY_VALS_PK" PRIMARY KEY ("PART_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."TYPES" ADD CONSTRAINT "TYPES_PK" PRIMARY KEY ("TYPES_ID"); - -ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "SQL110922153006740" PRIMARY KEY ("CD_ID", "COLUMN_NAME"); - -ALTER TABLE "APP"."PART_COL_PRIVS" ADD CONSTRAINT "PART_COL_PRIVS_PK" PRIMARY KEY ("PART_COLUMN_GRANT_ID"); - -ALTER TABLE "APP"."PARTITION_PARAMS" ADD CONSTRAINT "PARTITION_PARAMS_PK" PRIMARY KEY ("PART_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_PK" PRIMARY KEY ("PART_ID"); - -ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_PK" PRIMARY KEY ("TBL_ID", "PARAM_KEY"); - -ALTER TABLE "APP"."SKEWED_STRING_LIST" ADD CONSTRAINT "SKEWED_STRING_LIST_PK" PRIMARY KEY ("STRING_LIST_ID"); - -ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_PK" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_PK" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID"); - -ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_PK" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX"); - -ALTER TABLE "APP"."TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_PK" PRIMARY KEY ("CS_ID"); - -ALTER TABLE "APP"."PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_PK" PRIMARY KEY ("CS_ID"); - -ALTER TABLE "APP"."FUNCS" ADD CONSTRAINT "FUNCS_PK" PRIMARY KEY ("FUNC_ID"); - -ALTER TABLE "APP"."FUNC_RU" ADD CONSTRAINT "FUNC_RU_PK" PRIMARY KEY ("FUNC_ID", "INTEGER_IDX"); - -ALTER TABLE "APP"."NOTIFICATION_LOG" ADD CONSTRAINT "NOTIFICATION_LOG_PK" PRIMARY KEY ("NL_ID"); - -ALTER TABLE "APP"."NOTIFICATION_SEQUENCE" ADD CONSTRAINT "NOTIFICATION_SEQUENCE_PK" PRIMARY KEY ("NNI_ID"); - -ALTER TABLE "APP"."NOTIFICATION_SEQUENCE" ADD CONSTRAINT "ONE_ROW_CONSTRAINT" CHECK (NNI_ID = 1); - -ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("PARENT_TBL_ID", "CONSTRAINT_NAME", "POSITION"); - -ALTER TABLE "APP"."METASTORE_DB_PROPERTIES" ADD CONSTRAINT "PROPERTY_KEY_PK" PRIMARY KEY ("PROPERTY_KEY"); - -ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID"); - -ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID"); - - --- foreign -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK3" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_FK1" FOREIGN KEY ("INDEX_ID") REFERENCES "APP"."IDXS" ("INDEX_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEYS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK2" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."COLUMNS" ADD CONSTRAINT "COLUMNS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TYPE_FIELDS" ADD CONSTRAINT "TYPE_FIELDS_FK1" FOREIGN KEY ("TYPE_NAME") REFERENCES "APP"."TYPES" ("TYPES_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TBL_PRIVS" ADD CONSTRAINT "TBL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_FK1" FOREIGN KEY ("ROLE_ID") REFERENCES "APP"."ROLES" ("ROLE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PARTITION_KEY_VALS" ADD CONSTRAINT "PARTITION_KEY_VALS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "COLUMNS_V2_FK1" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PART_COL_PRIVS" ADD CONSTRAINT "PART_COL_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PARTITION_PARAMS" ADD CONSTRAINT "PARTITION_PARAMS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_FK1" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK1" FOREIGN KEY ("SD_ID_OID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK2" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_FK" FOREIGN KEY ("TBL_ID") REFERENCES TBLS("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_FK" FOREIGN KEY ("PART_ID") REFERENCES PARTITIONS("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."VERSION" ADD CONSTRAINT "VERSION_PK" PRIMARY KEY ("VER_ID"); - -ALTER TABLE "APP"."FUNCS" ADD CONSTRAINT "FUNCS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."FUNC_RU" ADD CONSTRAINT "FUNC_RU_FK1" FOREIGN KEY ("FUNC_ID") REFERENCES "APP"."FUNCS" ("FUNC_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_RESOURCEPLAN" ADD CONSTRAINT "WM_RESOURCEPLAN_PK" PRIMARY KEY ("RP_ID"); - -ALTER TABLE "APP"."WM_POOL" ADD CONSTRAINT "WM_POOL_PK" PRIMARY KEY ("POOL_ID"); - -ALTER TABLE "APP"."WM_POOL" ADD CONSTRAINT "WM_POOL_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_RESOURCEPLAN" ADD CONSTRAINT "WM_RESOURCEPLAN_FK1" FOREIGN KEY ("DEFAULT_POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_TRIGGER" ADD CONSTRAINT "WM_TRIGGER_PK" PRIMARY KEY ("TRIGGER_ID"); - -ALTER TABLE "APP"."WM_TRIGGER" ADD CONSTRAINT "WM_TRIGGER_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_POOL_TO_TRIGGER" ADD CONSTRAINT "WM_POOL_TO_TRIGGER_FK1" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_POOL_TO_TRIGGER" ADD CONSTRAINT "WM_POOL_TO_TRIGGER_FK2" FOREIGN KEY ("TRIGGER_ID") REFERENCES "APP"."WM_TRIGGER" ("TRIGGER_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_PK" PRIMARY KEY ("MAPPING_ID"); - -ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_PK" PRIMARY KEY ("TBL_ID", "MV_CREATION_METADATA_ID"); - -ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; - --- ---------------------------------------------- --- DDL Statements for checks --- ---------------------------------------------- - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "SQL110318025504980" CHECK (DEFERRED_REBUILD IN ('Y','N')); - -ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); - --- ---------------------------- --- Transaction and Lock Tables --- ---------------------------- -CREATE TABLE TXNS ( - TXN_ID bigint PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, - TXN_STATE char(1) NOT NULL, - TXN_STARTED bigint NOT NULL, - TXN_LAST_HEARTBEAT bigint NOT NULL, - TXN_USER varchar(128) NOT NULL, - TXN_HOST varchar(128) NOT NULL, - TXN_AGENT_INFO varchar(128), - TXN_META_INFO varchar(128), - TXN_HEARTBEAT_COUNT integer, - TXN_TYPE integer -); - -INSERT INTO TXNS (TXN_ID, TXN_STATE, TXN_STARTED, TXN_LAST_HEARTBEAT, TXN_USER, TXN_HOST) - VALUES(0, 'c', 0, 0, '', ''); - -CREATE TABLE TXN_COMPONENTS ( - TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID), - TC_DATABASE varchar(128) NOT NULL, - TC_TABLE varchar(256), - TC_PARTITION varchar(767), - TC_OPERATION_TYPE char(1) NOT NULL, - TC_WRITEID bigint -); - -CREATE INDEX TC_TXNID_INDEX ON TXN_COMPONENTS (TC_TXNID); - -CREATE TABLE COMPLETED_TXN_COMPONENTS ( - CTC_TXNID bigint NOT NULL, - CTC_DATABASE varchar(128) NOT NULL, - CTC_TABLE varchar(256), - CTC_PARTITION varchar(767), - CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL, - CTC_WRITEID bigint, - CTC_UPDATE_DELETE char(1) NOT NULL -); - -CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION); - -CREATE TABLE TXN_LOCK_TBL ( - TXN_LOCK bigint NOT NULL -); -INSERT INTO TXN_LOCK_TBL VALUES(1); - -CREATE TABLE HIVE_LOCKS ( - HL_LOCK_EXT_ID bigint NOT NULL, - HL_LOCK_INT_ID bigint NOT NULL, - HL_TXNID bigint NOT NULL, - HL_DB varchar(128) NOT NULL, - HL_TABLE varchar(256), - HL_PARTITION varchar(767), - HL_LOCK_STATE char(1) NOT NULL, - HL_LOCK_TYPE char(1) NOT NULL, - HL_LAST_HEARTBEAT bigint NOT NULL, - HL_ACQUIRED_AT bigint, - HL_USER varchar(128) NOT NULL, - HL_HOST varchar(128) NOT NULL, - HL_HEARTBEAT_COUNT integer, - HL_AGENT_INFO varchar(128), - HL_BLOCKEDBY_EXT_ID bigint, - HL_BLOCKEDBY_INT_ID bigint, - PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) -); - -CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); - -CREATE TABLE NEXT_LOCK_ID ( - NL_NEXT bigint NOT NULL -); -INSERT INTO NEXT_LOCK_ID VALUES(1); - -CREATE TABLE COMPACTION_QUEUE ( - CQ_ID bigint PRIMARY KEY, - CQ_DATABASE varchar(128) NOT NULL, - CQ_TABLE varchar(256) NOT NULL, - CQ_PARTITION varchar(767), - CQ_STATE char(1) NOT NULL, - CQ_TYPE char(1) NOT NULL, - CQ_TBLPROPERTIES varchar(2048), - CQ_WORKER_ID varchar(128), - CQ_ENQUEUE_TIME bigint, - CQ_START bigint, - CQ_RUN_AS varchar(128), - CQ_HIGHEST_WRITE_ID bigint, - CQ_META_INFO varchar(2048) for bit data, - CQ_HADOOP_JOB_ID varchar(32), - CQ_ERROR_MESSAGE clob, - CQ_NEXT_TXN_ID bigint, - CQ_TXN_ID bigint, - CQ_COMMIT_TIME bigint, - CQ_INITIATOR_ID varchar(128), - CQ_INITIATOR_VERSION varchar(128), - CQ_WORKER_VERSION varchar(128), - CQ_CLEANER_START bigint, - CQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0 -); - -CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( - NCQ_NEXT bigint NOT NULL -); -INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); - -CREATE TABLE COMPLETED_COMPACTIONS ( - CC_ID bigint PRIMARY KEY, - CC_DATABASE varchar(128) NOT NULL, - CC_TABLE varchar(256) NOT NULL, - CC_PARTITION varchar(767), - CC_STATE char(1) NOT NULL, - CC_TYPE char(1) NOT NULL, - CC_TBLPROPERTIES varchar(2048), - CC_WORKER_ID varchar(128), - CC_ENQUEUE_TIME bigint, - CC_START bigint, - CC_END bigint, - CC_RUN_AS varchar(128), - CC_HIGHEST_WRITE_ID bigint, - CC_META_INFO varchar(2048) for bit data, - CC_HADOOP_JOB_ID varchar(32), - CC_ERROR_MESSAGE clob, - CC_NEXT_TXN_ID bigint, - CC_TXN_ID bigint, - CC_COMMIT_TIME bigint, - CC_INITIATOR_ID varchar(128), - CC_INITIATOR_VERSION varchar(128), - CC_WORKER_VERSION varchar(128) -); - -CREATE INDEX COMPLETED_COMPACTIONS_RES ON COMPLETED_COMPACTIONS (CC_DATABASE,CC_TABLE,CC_PARTITION); - --- HIVE-25842 -CREATE TABLE COMPACTION_METRICS_CACHE ( - CMC_DATABASE varchar(128) NOT NULL, - CMC_TABLE varchar(256) NOT NULL, - CMC_PARTITION varchar(767), - CMC_METRIC_TYPE varchar(128) NOT NULL, - CMC_METRIC_VALUE integer NOT NULL, - CMC_VERSION integer NOT NULL -); - -CREATE TABLE AUX_TABLE ( - MT_KEY1 varchar(128) NOT NULL, - MT_KEY2 bigint NOT NULL, - MT_COMMENT varchar(255), - PRIMARY KEY(MT_KEY1, MT_KEY2) -); - ---1st 4 cols make up a PK but since WS_PARTITION is nullable we can't declare such PK ---This is a good candidate for Index orgainzed table -CREATE TABLE WRITE_SET ( - WS_DATABASE varchar(128) NOT NULL, - WS_TABLE varchar(256) NOT NULL, - WS_PARTITION varchar(767), - WS_TXNID bigint NOT NULL, - WS_COMMIT_ID bigint NOT NULL, - WS_OPERATION_TYPE char(1) NOT NULL -); - -CREATE TABLE TXN_TO_WRITE_ID ( - T2W_TXNID bigint NOT NULL, - T2W_DATABASE varchar(128) NOT NULL, - T2W_TABLE varchar(256) NOT NULL, - T2W_WRITEID bigint NOT NULL -); - -CREATE UNIQUE INDEX TBL_TO_TXN_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID); -CREATE UNIQUE INDEX TBL_TO_WRITE_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_WRITEID); - -CREATE TABLE NEXT_WRITE_ID ( - NWI_DATABASE varchar(128) NOT NULL, - NWI_TABLE varchar(256) NOT NULL, - NWI_NEXT bigint NOT NULL -); - -CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE); - -CREATE TABLE MIN_HISTORY_LEVEL ( - MHL_TXNID bigint NOT NULL, - MHL_MIN_OPEN_TXNID bigint NOT NULL, - PRIMARY KEY(MHL_TXNID) -); - -CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID); - -CREATE TABLE MATERIALIZATION_REBUILD_LOCKS ( - MRL_TXN_ID BIGINT NOT NULL, - MRL_DB_NAME VARCHAR(128) NOT NULL, - MRL_TBL_NAME VARCHAR(256) NOT NULL, - MRL_LAST_HEARTBEAT BIGINT NOT NULL, - PRIMARY KEY(MRL_TXN_ID) -); - -CREATE TABLE "APP"."I_SCHEMA" ( - "SCHEMA_ID" bigint primary key, - "SCHEMA_TYPE" integer not null, - "NAME" varchar(256) unique, - "DB_ID" bigint references "APP"."DBS" ("DB_ID"), - "COMPATIBILITY" integer not null, - "VALIDATION_LEVEL" integer not null, - "CAN_EVOLVE" char(1) not null, - "SCHEMA_GROUP" varchar(256), - "DESCRIPTION" varchar(4000) -); - -CREATE TABLE "APP"."SCHEMA_VERSION" ( - "SCHEMA_VERSION_ID" bigint primary key, - "SCHEMA_ID" bigint references "APP"."I_SCHEMA" ("SCHEMA_ID"), - "VERSION" integer not null, - "CREATED_AT" bigint not null, - "CD_ID" bigint references "APP"."CDS" ("CD_ID"), - "STATE" integer not null, - "DESCRIPTION" varchar(4000), - "SCHEMA_TEXT" clob, - "FINGERPRINT" varchar(256), - "SCHEMA_VERSION_NAME" varchar(256), - "SERDE_ID" bigint references "APP"."SERDES" ("SERDE_ID") -); - -CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SCHEMA_ID", "VERSION"); - -CREATE TABLE TXN_WRITE_NOTIFICATION_LOG ( - WNL_ID bigint NOT NULL, - WNL_TXNID bigint NOT NULL, - WNL_WRITEID bigint NOT NULL, - WNL_DATABASE varchar(128) NOT NULL, - WNL_TABLE varchar(256) NOT NULL, - WNL_PARTITION varchar(767) NOT NULL, - WNL_TABLE_OBJ clob NOT NULL, - WNL_PARTITION_OBJ clob, - WNL_FILES clob, - WNL_EVENT_TIME integer NOT NULL, - PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION) -); -INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1); - --- ----------------------------------------------------------------- --- Record schema version. Should be the last step in the init script --- ----------------------------------------------------------------- -INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '4.0.0-alpha-2', 'Hive release version 4.0.0-alpha-2'); diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/pom.xml deleted file mode 100644 index db24c40564..0000000000 --- a/nifi-extension-bundles/nifi-iceberg-bundle/pom.xml +++ /dev/null @@ -1,173 +0,0 @@ - - - - - org.apache.nifi - nifi-standard-services-api-bom - 2.0.0-SNAPSHOT - ../nifi-standard-services-api-bom - - 4.0.0 - - nifi-iceberg-bundle - pom - - - 1.6.1 - 3.1.3 - - - - nifi-iceberg-services-api - nifi-iceberg-services-api-nar - nifi-iceberg-services - nifi-iceberg-services-nar - nifi-iceberg-processors - nifi-iceberg-processors-nar - nifi-iceberg-common - nifi-iceberg-test-utils - - - - - - io.netty - netty - ${netty.3.version} - - - - org.glassfish - javax.el - 3.0.1-b12 - - - - commons-beanutils - commons-beanutils - 1.9.4 - - - - org.apache.derby - derby - ${derby.version} - - - - org.apache.ant - ant - 1.10.15 - - - - org.codehaus.jettison - jettison - 1.5.4 - - - - com.nimbusds - nimbus-jose-jwt - 9.41.2 - - - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - ${hadoop.version} - - - org.bouncycastle - bcprov-jdk15on - - - org.bouncycastle - bcpkix-jdk15on - - - org.eclipse.jetty - jetty-util-ajax - - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.slf4j - slf4j-reload4j - - - commons-logging - commons-logging - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-webapp - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-http - - - org.eclipse.jetty - jetty-rewrite - - - - org.bouncycastle - bcprov-jdk15on - - - - - - com.google.guava - guava - 33.3.1-jre - - - - org.codehaus.groovy - groovy-all - 2.4.21 - - - - diff --git a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services-nar/src/main/resources/META-INF/NOTICE b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services-nar/src/main/resources/META-INF/NOTICE index b57ec55464..1d9f0842ac 100644 --- a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services-nar/src/main/resources/META-INF/NOTICE +++ b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services-nar/src/main/resources/META-INF/NOTICE @@ -1,4 +1,4 @@ -nifi-iceberg-processors-nar +nifi-protobuf-services-nar Copyright 2014-2023 The Apache Software Foundation This product includes software developed at diff --git a/nifi-extension-bundles/pom.xml b/nifi-extension-bundles/pom.xml index b125514a09..87e58db751 100755 --- a/nifi-extension-bundles/pom.xml +++ b/nifi-extension-bundles/pom.xml @@ -84,7 +84,6 @@ nifi-box-bundle nifi-flow-registry-client-bundle nifi-shopify-bundle - nifi-iceberg-bundle nifi-jslt-bundle nifi-iotdb-bundle nifi-cipher-bundle