You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/11/04 11:30:38 UTC

[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6584: NIFI-10370 Create record oriented PutSnowflake processor

krisztina-zsihovszki commented on code in PR #6584:
URL: https://github.com/apache/nifi/pull/6584#discussion_r1013902699


##########
nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-processors/src/main/java/org/apache/nifi/processors/snowflake/StartSnowflakeIngest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.snowflake;
+
+import static org.apache.nifi.processors.snowflake.common.Attributes.ATTRIBUTE_STAGED_FILE_PATH;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import net.snowflake.ingest.SimpleIngestManager;
+import net.snowflake.ingest.connection.IngestResponseException;
+import net.snowflake.ingest.utils.StagedFileWrapper;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.ReadsAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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;
+
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@ReadsAttributes({
+        @ReadsAttribute(attribute = ATTRIBUTE_STAGED_FILE_PATH, description = "The path to the file in the stage")
+})
+@Tags({"snowflake", "snowpipe", "ingest"})
+@CapabilityDescription("Ingest files in a Snowflake stage. The stage must be created in the Snowflake account beforehand."
+        + " The result of the ingestion is not available immediately, so this processor can be connected to an"
+        + " GetSnowflakeIngestStatus processor to wait for the results")
+@SeeAlso({PutSnowflakeInternalStage.class, GetSnowflakeIngestStatus.class})
+public class StartSnowflakeIngest extends AbstractProcessor {
+
+    static final PropertyDescriptor INGEST_MANAGER_PROVIDER = new PropertyDescriptor.Builder()
+            .name("ingest-manager-provider")
+            .displayName("Ingest Manager Provider")
+            .description("Ingest manager provider")
+            .identifiesControllerService(SnowflakeIngestManagerProviderService.class)
+            .required(true)
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles of successful ingest request")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("For FlowFiles of failed ingest request")
+            .build();
+
+    static final List<PropertyDescriptor> PROPERTIES = Collections.singletonList(
+            INGEST_MANAGER_PROVIDER
+    );
+
+    static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        final SnowflakeIngestManagerProviderService ingestManagerProviderService =
+                context.getProperty(INGEST_MANAGER_PROVIDER)
+                        .asControllerService(SnowflakeIngestManagerProviderService.class);
+        final FlowFile flowFile = session.get();
+        final SimpleIngestManager snowflakeIngestManager = ingestManagerProviderService.getIngestManager();
+        final String stagedFileName = flowFile.getAttribute(ATTRIBUTE_STAGED_FILE_PATH);
+        final StagedFileWrapper stagedFile = new StagedFileWrapper(stagedFileName);

Review Comment:
   Please consider checking whether the "snowflake.staged.file.path" is really set on the incoming flowfile to avoid possible NullPointerException. 



##########
nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-processors/src/main/java/org/apache/nifi/processors/snowflake/GetSnowflakeIngestStatus.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.snowflake;
+
+import static org.apache.nifi.processors.snowflake.common.Attributes.ATTRIBUTE_STAGED_FILE_PATH;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import net.snowflake.ingest.SimpleIngestManager;
+import net.snowflake.ingest.connection.HistoryResponse;
+import net.snowflake.ingest.connection.HistoryResponse.FileEntry;
+import net.snowflake.ingest.connection.IngestResponseException;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.ReadsAttributes;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.flowfile.FlowFile;
+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;
+
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Stateful(scopes = Scope.CLUSTER,
+        description = "The 'begin mark' from the response of a history request is stored to keep track of already requested history time range.")
+@DefaultSettings(penaltyDuration = "5 sec")
+@ReadsAttributes({
+        @ReadsAttribute(attribute = ATTRIBUTE_STAGED_FILE_PATH, description = "The path to the file in the stage")
+})
+@Tags({"snowflake", "snowpipe", "ingest", "history"})
+@CapabilityDescription("Waits until a file in a Snowflake stage is ingested. The stage must be created in the Snowflake account beforehand."
+        + " This processor is usually connected to an upstream StartSnowflakeIngest processor to make sure that the file is ingested")
+@SeeAlso({StartSnowflakeIngest.class, PutSnowflakeInternalStage.class})
+public class GetSnowflakeIngestStatus extends AbstractProcessor {
+
+    static final PropertyDescriptor INGEST_MANAGER_PROVIDER = new PropertyDescriptor.Builder()
+            .name("ingest-manager-provider")
+            .displayName("Ingest Manager Provider")
+            .description("Ingest manager provider")
+            .identifiesControllerService(SnowflakeIngestManagerProviderService.class)
+            .required(true)
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles of successful ingestion")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("For FlowFiles of failed ingestion")
+            .build();
+
+    static final Relationship REL_RETRY = new Relationship.Builder()
+            .name("retry")
+            .description("For FlowFiles whose file is still not ingested. These FlowFiles should be routed back to this processor to try again later")
+            .build();
+
+    static final List<PropertyDescriptor> PROPERTIES = Collections.singletonList(
+            INGEST_MANAGER_PROVIDER
+    );
+
+    static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_RETRY);
+        relationships.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        final SnowflakeIngestManagerProviderService ingestManagerProviderService =
+                context.getProperty(INGEST_MANAGER_PROVIDER)
+                        .asControllerService(SnowflakeIngestManagerProviderService.class);
+
+        final FlowFile flowFile = session.get();
+        final String stagedFilePath = flowFile.getAttribute(ATTRIBUTE_STAGED_FILE_PATH);
+        final String beginMarkKey = stagedFilePath + ".begin.mark";
+        final StateManager stateManager = StateManager.create(beginMarkKey, session);
+        final String beginMark = stateManager.getBeginMark();
+        final HistoryResponse historyResponse;
+        try {
+            final SimpleIngestManager snowflakeIngestManager = ingestManagerProviderService.getIngestManager();
+            historyResponse = snowflakeIngestManager.getHistory(null, null, beginMark);
+        } catch (URISyntaxException | IOException e) {
+            throw new ProcessException("Failed to get Snowflake ingest history for staged file [" + stagedFilePath + "]", e);
+        } catch (IngestResponseException e) {
+            getLogger().error("Failed to get Snowflake ingest history for staged file [" + stagedFilePath + "]", e);
+            session.transfer(session.penalize(flowFile), REL_FAILURE);
+            return;
+        }
+
+        final Optional<FileEntry> fileEntry = Optional.ofNullable(historyResponse.files)
+                .flatMap(files -> files.stream()
+                        .filter(entry -> entry.getPath().equals(stagedFilePath) && entry.isComplete())
+                        .findFirst());
+
+        if (!fileEntry.isPresent()) {
+            stateManager.saveBeginMarkToState(historyResponse.getNextBeginMark());
+            session.transfer(session.penalize(flowFile), REL_RETRY);
+            return;
+        }
+
+        stateManager.removeBeginMarkFromState();
+        if (fileEntry.get().getErrorsSeen() > 0) {
+            getLogger().error("Failed to ingest file [" + stagedFilePath + "] in Snowflake stage via pipe [" + ingestManagerProviderService.getPipeName() + "]."
+                    + " Error: " + fileEntry.get().getFirstError());
+            session.transfer(session.penalize(flowFile), REL_FAILURE);

Review Comment:
   Return statement is missing after transferring the flowfile to FAILURE.



##########
nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/StandardSnowflakeIngestManagerProviderService.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.snowflake.service;
+
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.spec.InvalidKeySpecException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import net.snowflake.ingest.SimpleIngestManager;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.key.service.api.PrivateKeyService;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.snowflake.SnowflakeIngestManagerProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.snowflake.service.util.AccountIdentifierFormat;
+import org.apache.nifi.snowflake.service.util.CommonProperties;
+
+@Tags({"snowflake", "jdbc", "database", "connection"})
+@CapabilityDescription("Provides a Snowflake Ingest Manager for Snowflake pipe processors")
+public class StandardSnowflakeIngestManagerProviderService extends AbstractControllerService

Review Comment:
   Please consider adding a StandardProxyConfigurationService to the processors or to the controller services to handle the case when NiFi is behind a proxy.



##########
nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-processors/src/main/resources/docs/org.apache.nifi.processors.snowflake.PutSnowflakeInternalStage/additionalDetails.html:
##########
@@ -0,0 +1,42 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8"/>
+    <title>PutSnowflakeInternalStage</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>
+        h2 {margin-top: 4em}
+        h3 {margin-top: 3em}
+        td {text-align: left}
+    </style>
+</head>
+<body>
+
+<h1>PutSnowflakeInternalStage</h1>
+
+<h3>Description</h3>
+<p>
+    The PutSnowflakeInternalStage processor can upload a file to a Snowflake internal stage. This stage needs to be set up in your Snowflake account.
+    The processor requires an upstream connection and the incoming FlowFiles content will be uploaded to the stage. The attributes "filename" and "path" are used to provide a prefix and file name for your file in the stage.
+    While the processor can be used by itself, it's usually recommended to connect it to an StartSnowflakeIngest processor to put it into your Snowflake table via a pipe.

Review Comment:
   The "it" refers to the processor first then the to the file. You can rephrase it like: 
   "While the processor can be used by itself, it's usually recommended to connect it to a StartSnowflakeIngest processor to put the **uploaded file** into your Snowflake table via a pipe" 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org