[ 
https://issues.apache.org/jira/browse/NIFI-238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14324561#comment-14324561
 ] 

ASF GitHub Bot commented on NIFI-238:
-------------------------------------

Github user rdblue commented on a diff in the pull request:

    https://github.com/apache/incubator-nifi/pull/24#discussion_r24836941
  
    --- Diff: 
nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/KiteStorageProcessor.java
 ---
    @@ -0,0 +1,169 @@
    +/*
    + * 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.kite;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableSet;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericData.Record;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.FlowFileAccessException;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.InputStreamCallback;
    +import org.apache.nifi.util.LongHolder;
    +import org.apache.nifi.util.StopWatch;
    +import org.kitesdk.data.DatasetIOException;
    +import org.kitesdk.data.DatasetWriter;
    +import org.kitesdk.data.Datasets;
    +import org.kitesdk.data.IncompatibleSchemaException;
    +import org.kitesdk.data.ValidationException;
    +import org.kitesdk.data.View;
    +import org.kitesdk.data.spi.SchemaValidationUtil;
    +
    +public class KiteStorageProcessor extends AbstractKiteProcessor {
    +  private static Relationship SUCCESS = new Relationship.Builder()
    +      .name("success")
    +      .description("FlowFile content has been successfully saved")
    +      .build();
    +
    +  private static Relationship INCOMPATIBLE = new Relationship.Builder()
    +      .name("incompatible")
    +      .description("FlowFile content is not compatible with the target 
dataset")
    +      .build();
    +
    +  private static Relationship FAILURE = new Relationship.Builder()
    +      .name("failure")
    +      .description("FlowFile content could not be processed")
    +      .build();
    +
    +  public static final PropertyDescriptor KITE_DATASET_URI =
    +      new PropertyDescriptor.Builder()
    +          .name("Target dataset URI")
    +          .description(
    +              "URI that identifies a Kite dataset where data will be 
stored")
    +          .addValidator(RECOGNIZED_URI)
    +          .expressionLanguageSupported(true)
    +          .required(true)
    +          .build();
    +
    +  private static final List<PropertyDescriptor> PROPERTIES =
    +      ImmutableList.<PropertyDescriptor>builder()
    +          .addAll(AbstractKiteProcessor.getProperties())
    +          .add(KITE_DATASET_URI)
    +          .build();
    +
    +  private static final Set<Relationship> RELATIONSHIPS =
    +      ImmutableSet.<Relationship>builder()
    +          .add(SUCCESS)
    +          .add(INCOMPATIBLE)
    +          .add(FAILURE)
    +          .build();
    +
    +  @Override
    +  protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +    return PROPERTIES;
    +  }
    +
    +  @Override
    +  public Set<Relationship> getRelationships() {
    +    return RELATIONSHIPS;
    +  }
    +
    +  @Override
    +  public void onTrigger(ProcessContext context, ProcessSession session)
    +      throws ProcessException {
    +    FlowFile flowFile = session.get();
    +    if (flowFile == null) {
    +      return;
    +    }
    +
    +    final View<Record> target = load(context, flowFile);
    +    final Schema schema = target.getDataset().getDescriptor().getSchema();
    +
    +    try {
    +      final LongHolder writtenCount = new LongHolder(0L);
    +      StopWatch timer = new StopWatch(true);
    +      session.read(flowFile, new InputStreamCallback() {
    +        @Override
    +        public void process(InputStream in) throws IOException {
    +          try (DataFileStream<Record> stream = new DataFileStream<>(
    +              in, AvroUtil.newDatumReader(schema, Record.class))) {
    +            IncompatibleSchemaException.check(
    +                SchemaValidationUtil.canRead(stream.getSchema(), schema),
    +                "Incompatible file schema %s, expected %s",
    +                stream.getSchema(), schema);
    +
    +            long written = 0L;
    +            try (DatasetWriter<Record> writer = target.newWriter()) {
    +              for (Record record : stream) {
    +                writer.write(record);
    +                written += 1;
    +              }
    +            } finally {
    +              writtenCount.set(written);
    +            }
    +          }
    +        }
    +      });
    +      timer.stop();
    +
    +      session.adjustCounter("Stored records", writtenCount.get(),
    --- End diff --
    
    This needs to be in the finally block because the records are written at 
that point and an exception will set the count, but skip updating the counter.


> Add processors to write datasets using Kite
> -------------------------------------------
>
>                 Key: NIFI-238
>                 URL: https://issues.apache.org/jira/browse/NIFI-238
>             Project: Apache NiFi
>          Issue Type: New Feature
>          Components: Extensions
>            Reporter: Ryan Blue
>
> I think it would be great to have a set of processors that parse incoming 
> flow files and add the data to Kite datasets.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to