Github user ijokarumawak commented on a diff in the pull request:
https://github.com/apache/nifi/pull/2737#discussion_r194217227
--- Diff:
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RecordStats.java
---
@@ -0,0 +1,225 @@
+/*
+ * 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.standard;
+
+import com.google.common.collect.Lists;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.InputStream;
+import java.util.ArrayList;
+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 java.util.stream.Collectors;
+
+@Tags({ "record", "stats", "metrics" })
+@CapabilityDescription("A processor that can count the number of items in
a record set, as well as provide counts based on " +
+ "user-defined criteria on subsets of the record set.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@WritesAttributes({
+ @WritesAttribute(attribute = RecordStats.RECORD_COUNT_ATTR,
description = "A count of the records in the record set in the flowfile.")
+})
+public class RecordStats extends AbstractProcessor {
+ static final String RECORD_COUNT_ATTR = "record_count";
+
+ static final PropertyDescriptor RECORD_READER = new
PropertyDescriptor.Builder()
+ .name("record-stats-reader")
+ .displayName("Record Reader")
+ .description("A record reader to use for reading the records.")
+ .identifiesControllerService(RecordReaderFactory.class)
+ .build();
+
+ static final PropertyDescriptor LIMIT = new
PropertyDescriptor.Builder()
+ .name("record-stats-limit")
+ .description("Limit the number of individual stats that are
returned for each record path to the top N results.")
+ .required(true)
+ .defaultValue("10")
+ .addValidator(StandardValidators.INTEGER_VALIDATOR)
+
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+ .build();
+
+ static final Relationship REL_SUCCESS = new Relationship.Builder()
+ .name("success")
+ .description("If a flowfile is successfully processed, it goes
here.")
+ .build();
+ static final Relationship REL_FAILURE = new Relationship.Builder()
+ .name("failure")
+ .description("If a flowfile fails to be processed, it goes here.")
+ .build();
+
+ private RecordPathCache cache;
+
+ static final Set RELATIONSHIPS;
+ static final List<PropertyDescriptor> PROPERTIES;
+
+ static {
+ Set _rels = new HashSet();
+ _rels.add(REL_SUCCESS);
+ _rels.add(REL_FAILURE);
+ RELATIONSHIPS = Collections.unmodifiableSet(_rels);
+ List<PropertyDescriptor> _temp = new ArrayList<>();
+ _temp.add(RECORD_READER);
+ _temp.add(LIMIT);
+ PROPERTIES = Collections.unmodifiableList(_temp);
+ }
+
+ protected PropertyDescriptor
getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+ return new PropertyDescriptor.Builder()
+ .name(propertyDescriptorName)
+ .displayName(propertyDescriptorName)
+ .dynamic(true)
+ .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+ .build();
+ }
+
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ return PROPERTIES;
+ }
+
+ @OnScheduled
+ public void onEnabled(ProcessContext context) {
+ cache = new RecordPathCache(25);
+ }
+
+ @Override
+ public Set<Relationship> getRelationships() {
+ return RELATIONSHIPS;
+ }
+
+ @Override
+ public void onTrigger(ProcessContext context, ProcessSession session)
throws ProcessException {
+ FlowFile input = session.get();
+ if (input == null) {
+ return;
+ }
+
+ try {
+ Map<String, RecordPath> paths = getRecordPaths(context);
+ Map<String, String> stats = getStats(input, paths, context,
session);
+
+ input = session.putAllAttributes(input, stats);
+
+ session.transfer(input, REL_SUCCESS);
+
+ } catch (Exception ex) {
+ getLogger().error("Error processing stats.", ex);
+ session.transfer(input, REL_FAILURE);
+ }
+
+ }
+
+ protected Map<String, RecordPath> getRecordPaths(ProcessContext
context) {
+ return context.getProperties().keySet()
+ .stream().filter(p -> p.isDynamic() &&
!p.getName().contains(RECORD_READER.getName()))
--- End diff --
I think `!p.getName().contains(RECORD_READER.getName())` part is not
necessary.
---