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

    https://github.com/apache/nifi/pull/1961#discussion_r125645539
  
    --- Diff: 
nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseRecord.java
 ---
    @@ -0,0 +1,316 @@
    +/*
    + * 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.hbase;
    +
    +import org.apache.hadoop.hbase.util.Bytes;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.hbase.put.PutColumn;
    +import org.apache.nifi.hbase.put.PutFlowFile;
    +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.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordFieldType;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +@EventDriven
    +@SupportsBatching
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@Tags({"hadoop", "hbase", "put", "record"})
    +@CapabilityDescription("Adds rows to HBase based on the contents of a 
flowfile using a configured record reader.")
    +public class PutHBaseRecord extends AbstractPutHBase {
    +
    +    protected static final PropertyDescriptor ROW_FIELD_NAME = new 
PropertyDescriptor.Builder()
    +            .name("Row Identifier Field Name")
    +            .description("Specifies the name of a JSON element whose value 
should be used as the row id for the given JSON document.")
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    protected static final String FAIL_VALUE = "Fail";
    +    protected static final String WARN_VALUE = "Warn";
    +    protected static final String IGNORE_VALUE = "Ignore";
    +    protected static final String TEXT_VALUE = "Text";
    +
    +    protected static final AllowableValue COMPLEX_FIELD_FAIL = new 
AllowableValue(FAIL_VALUE, FAIL_VALUE, "Route entire FlowFile to failure if any 
elements contain complex values.");
    +    protected static final AllowableValue COMPLEX_FIELD_WARN = new 
AllowableValue(WARN_VALUE, WARN_VALUE, "Provide a warning and do not include 
field in row sent to HBase.");
    +    protected static final AllowableValue COMPLEX_FIELD_IGNORE = new 
AllowableValue(IGNORE_VALUE, IGNORE_VALUE, "Silently ignore and do not include 
in row sent to HBase.");
    +    protected static final AllowableValue COMPLEX_FIELD_TEXT = new 
AllowableValue(TEXT_VALUE, TEXT_VALUE, "Use the string representation of the 
complex field as the value of the given column.");
    +
    +    static final PropertyDescriptor RECORD_READER_FACTORY = 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();
    +
    +    protected static final PropertyDescriptor COMPLEX_FIELD_STRATEGY = new 
PropertyDescriptor.Builder()
    +            .name("Complex Field Strategy")
    +            .description("Indicates how to handle complex fields, i.e. 
fields that do not have a single text value.")
    +            .expressionLanguageSupported(false)
    +            .required(true)
    +            .allowableValues(COMPLEX_FIELD_FAIL, COMPLEX_FIELD_WARN, 
COMPLEX_FIELD_IGNORE, COMPLEX_FIELD_TEXT)
    +            .defaultValue(COMPLEX_FIELD_TEXT.getValue())
    +            .build();
    +
    +
    +    protected static final AllowableValue FIELD_ENCODING_STRING = new 
AllowableValue(STRING_ENCODING_VALUE, STRING_ENCODING_VALUE,
    +            "Stores the value of each field as a UTF-8 String.");
    +    protected static final AllowableValue FIELD_ENCODING_BYTES = new 
AllowableValue(BYTES_ENCODING_VALUE, BYTES_ENCODING_VALUE,
    +            "Stores the value of each field as the byte representation of 
the type derived from the JSON.");
    +
    +    protected static final PropertyDescriptor FIELD_ENCODING_STRATEGY = 
new PropertyDescriptor.Builder()
    +            .name("Field Encoding Strategy")
    +            .description(("Indicates how to store the value of each field 
in HBase. The default behavior is to convert each value from the " +
    +                    "JSON to a String, and store the UTF-8 bytes. Choosing 
Bytes will interpret the type of each field from " +
    +                    "the JSON, and convert the value to the byte 
representation of that type, meaning an integer will be stored as the " +
    +                    "byte representation of that integer."))
    +            .required(true)
    +            .allowableValues(FIELD_ENCODING_STRING, FIELD_ENCODING_BYTES)
    +            .defaultValue(FIELD_ENCODING_STRING.getValue())
    +            .build();
    +
    +    @Override
    +    public final List<PropertyDescriptor> 
getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(RECORD_READER_FACTORY);
    +        properties.add(HBASE_CLIENT_SERVICE);
    +        properties.add(TABLE_NAME);
    +        properties.add(ROW_ID);
    +        properties.add(ROW_FIELD_NAME);
    +        properties.add(ROW_ID_ENCODING_STRATEGY);
    +        properties.add(COLUMN_FAMILY);
    +        properties.add(BATCH_SIZE);
    +        properties.add(COMPLEX_FIELD_STRATEGY);
    +        properties.add(FIELD_ENCODING_STRATEGY);
    +        return properties;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        final Set<Relationship> rels = new HashSet<>();
    +        rels.add(REL_SUCCESS);
    +        rels.add(REL_FAILURE);
    +        return rels;
    +    }
    +
    +
    +    private RecordReaderFactory recordParserFactory;
    +    @Override
    +    public void onTrigger(final ProcessContext context, final 
ProcessSession session) throws ProcessException {
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        List<FlowFile> flowFiles = session.get(batchSize);
    +        if (flowFiles == null || flowFiles.size() == 0) {
    +            return;
    +        }
    +
    +        final Map<String,List<PutFlowFile>> tablePuts = new HashMap<>();
    +        final RecordReaderFactory recordParserFactory = 
context.getProperty(RECORD_READER_FACTORY)
    +                .asControllerService(RecordReaderFactory.class);
    +
    +        // Group FlowFiles by HBase Table
    +
    +        final Map<FlowFile, List<PutFlowFile>> results = new HashMap<>();
    +
    +        for (final FlowFile flowFile : flowFiles) {
    +            getLogger().info("Working on flowfile with size {}", new 
Object[]{flowFile.getSize()});
    +            final List<PutFlowFile> puts = createPuts(session, context, 
flowFile, recordParserFactory);
    +
    +            if (puts.size() == 0) {
    +                getLogger().info("Put list was empty, skipping...");
    +                session.transfer(flowFile, REL_FAILURE);
    +                continue;
    +            }
    +
    +            boolean foundInvalid = false;
    +            for (PutFlowFile putFlowFile : puts) {
    +                if (!putFlowFile.isValid() && !foundInvalid) {
    +                    session.transfer(flowFile, REL_FAILURE);
    +                    foundInvalid = true;
    +                }
    +            }
    +
    +            if (!foundInvalid) {
    +                String tableName = puts.get(0).getTableName();
    +                List<PutFlowFile> putFlowFiles = tablePuts.get(tableName);
    +                if (putFlowFiles == null) {
    +                    putFlowFiles = new ArrayList<>();
    +                    tablePuts.put(tableName, putFlowFiles);
    +                }
    +                putFlowFiles.addAll(puts);
    +
    +                results.put(flowFile, puts);
    +            }
    +        }
    +
    +        getLogger().debug("Sending {} FlowFiles to HBase in {} put 
operations", new Object[]{flowFiles.size(), tablePuts.size()});
    +
    +        final long start = System.nanoTime();
    +        final List<FlowFile> successes = new ArrayList<>();
    +        final Map<FlowFile, Integer> columnsAdded = new HashMap<>();
    +        final Map<FlowFile, List<String>> provenanceEvents = new 
HashMap<>();
    +
    +        for (Map.Entry<FlowFile, List<PutFlowFile>> entry : 
results.entrySet()) {
    +            try {
    +                clientService.put(entry.getValue().get(0).getTableName(), 
entry.getValue());
    +                successes.add(entry.getKey());
    +
    +                int columns = 0;
    +                for (PutFlowFile putFlowFile : entry.getValue()) {
    +                    columns += putFlowFile.getColumns().size();
    +                }
    +                PutFlowFile first = entry.getValue().get(0);
    +                PutFlowFile last = 
entry.getValue().get(entry.getValue().size() - 1);
    +                List<String> events = Arrays.asList(getTransitUri(first), 
getTransitUri(last));
    +
    +                columnsAdded.put( entry.getKey(), new Integer(columns));
    +                provenanceEvents.put(entry.getKey(), events);
    +            } catch (Exception e) {
    +                getLogger().error(e.getMessage(), e);
    +
    +                final FlowFile failure = session.penalize(entry.getKey());
    +                session.transfer(failure, REL_FAILURE);
    +            }
    +        }
    +
    +        final long sendMillis = 
TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
    +        getLogger().debug("Sent {} FlowFiles to HBase successfully in {} 
milliseconds", new Object[]{successes.size(), sendMillis});
    +
    +        for (FlowFile flowFile : successes) {
    +            session.transfer(flowFile, REL_SUCCESS);
    +            final String details = "Put " + columnsAdded.get(flowFile) + " 
cells to HBase";
    +            session.getProvenanceReporter().send(flowFile, 
provenanceEvents.get(flowFile).get(0), details, sendMillis);
    +            session.getProvenanceReporter().send(flowFile, 
provenanceEvents.get(flowFile).get(0), details, sendMillis);
    +        }
    +
    +        session.commit();
    +    }
    +
    +
    +    @Override
    +    protected PutFlowFile createPut(ProcessSession session, ProcessContext 
context, FlowFile flowFile) {
    +        return null;
    +    }
    +
    +    protected byte[] asBytes(String field, Object input) {
    +        byte[] retVal = null;
    +
    +        if (input instanceof Number) {
    +            if (input instanceof Float || input instanceof Double) {
    +                retVal = 
clientService.toBytes(Double.parseDouble(input.toString()));
    +            } else {
    +                retVal = 
clientService.toBytes(Long.parseLong(input.toString()));
    +            }
    +        } else if (input instanceof Boolean) {
    +            retVal = clientService.toBytes((Boolean)input);
    +        } else if (input instanceof String) {
    +            retVal = clientService.toBytes((String)input);
    +        } else {
    +            throw new RuntimeException(String.format("Could not identify 
type for field %", field));
    +        }
    +
    +        return retVal;
    +    }
    +
    +    protected byte[] asBytes(String field, RecordFieldType fieldType, 
Record record, boolean asString) {
    +
    +        byte[] retVal;
    +
    +        if (asString) {
    +            retVal = Bytes.toBytes(record.getAsString(field));
    +        } else {
    +            switch (fieldType) {
    +                case BOOLEAN:
    +                    retVal = Bytes.toBytes(record.getAsBoolean(field));
    +                    break;
    +                case CHAR:
    +                    retVal = Bytes.toBytes(record.getAsString(field));
    +                    break;
    +                case DOUBLE:
    +                    retVal = Bytes.toBytes(record.getAsDouble(field));
    +                    break;
    +                case FLOAT:
    +                    retVal = Bytes.toBytes(record.getAsFloat(field));
    +                    break;
    +                case INT:
    +                    retVal = Bytes.toBytes(record.getAsInt(field));
    +                    break;
    +                case LONG:
    +                    retVal = Bytes.toBytes(record.getAsLong(field));
    +                    break;
    +                default:
    +                    retVal = Bytes.toBytes(record.getAsString(field));
    +            }
    +        }
    +
    +        return retVal;
    +    }
    +
    +    protected List<PutFlowFile> createPuts(ProcessSession session, 
ProcessContext context, FlowFile flowFile, RecordReaderFactory 
recordParserFactory) {
    --- End diff --
    
    I think we should try to avoid reading the entire flow file into memory if 
possible... 
    
    With PutHBaseJSON we had one JSON message per flow file, so the # of 
messages in memory at one time would be equal to the batch size. With the 
record oriented processors, the whole idea is to keep a lot of records together 
for efficiency, so we might have millions of records in a single flow file 
which could lead to memory issues if they were all read into memory.
    
    A possible approach to deal with this could be the following... 
    
    We could change the meaning of Batch Size in this processor, and instead of 
it being the number of flow files to grab, it could be the number of records to 
send to HBase in one call. So the processor grabs one flow file, creates a 
record reader and starts reading records up to batch size, sends the batch to 
HBase, and then starts a new batch. We can also keep track of the index of the 
last successful record, and if an error occurs at any point we then route the 
flow file to failure and add an attribute like 
"record.successful.index.<processor-id>". If the processor receives a flow file 
with this attribute already populated, then it will pick up from this index 
when sending records. 
    
    Thoughts?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to