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 2021/10/27 22:00:11 UTC

[GitHub] [nifi] exceptionfactory commented on a change in pull request #5476: NIFI-9259: Adding GeohashRecord Processor

exceptionfactory commented on a change in pull request #5476:
URL: https://github.com/apache/nifi/pull/5476#discussion_r737870081



##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";
+
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+            .name("mode")
+            .displayName("Mode")
+            .description("Specifies whether to encode latitude/longitude to geohash or decode geohash to latitude/longitude")
+            .required(true)
+            .allowableValues(ENCODE_MODE, DECODE_MODE)
+            .defaultValue(ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the record reader service to use for reading incoming data")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the record writer service to use for writing data")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    public static final PropertyDescriptor LATITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("latitude-record-path")
+            .displayName("Latitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the latitude value;"
+                    + "in the decode mode, this property specifies the record path to put the latitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor LONGITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("longitude-record-path")
+            .displayName("Longitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the longitude value;"
+                    + "in the decode mode, this property specifies the record path to put the longitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_FORMAT = new PropertyDescriptor.Builder()
+            .name("geohash-format")
+            .displayName("Geohash Format")
+            .description("In the encode mode, this property specifies the desired format for encoding geohash;"
+                    + "in the decode mode, this property specifies the format of geohash provided")
+            .required(true)
+            .allowableValues(BASE32, BINARY_STRING, LONG_INT)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_LEVEL = new PropertyDescriptor.Builder()
+            .name("geohash-level")
+            .displayName("Geohash Level")
+            .description("The integer precision level(1-12) desired for encoding geohash")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dependsOn(MODE, ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("geohash-record-path")
+            .displayName("Geohash Record Path")
+            .description("In the encode mode, this property specifies the record path to put the geohash value;"
+                    + "in the decode mode, this property specifies the record path to retrieve the geohash value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Records that are successfully encoded or decoded will be routed to success")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Records that cannot be encoded or decoded will be routed to failure")
+            .build();
+
+    public static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(

Review comment:
       It looks like this could be changed to `private`.
   ```suggestion
       private static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(
   ```

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";

Review comment:
       Recommend encapsulated these values in a separate `enum` named something like `GeohashFormat`, which could be used directly in `allowableValues()`.

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";
+
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+            .name("mode")
+            .displayName("Mode")
+            .description("Specifies whether to encode latitude/longitude to geohash or decode geohash to latitude/longitude")
+            .required(true)
+            .allowableValues(ENCODE_MODE, DECODE_MODE)
+            .defaultValue(ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the record reader service to use for reading incoming data")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the record writer service to use for writing data")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    public static final PropertyDescriptor LATITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("latitude-record-path")
+            .displayName("Latitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the latitude value;"
+                    + "in the decode mode, this property specifies the record path to put the latitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor LONGITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("longitude-record-path")
+            .displayName("Longitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the longitude value;"
+                    + "in the decode mode, this property specifies the record path to put the longitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_FORMAT = new PropertyDescriptor.Builder()
+            .name("geohash-format")
+            .displayName("Geohash Format")
+            .description("In the encode mode, this property specifies the desired format for encoding geohash;"
+                    + "in the decode mode, this property specifies the format of geohash provided")
+            .required(true)
+            .allowableValues(BASE32, BINARY_STRING, LONG_INT)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_LEVEL = new PropertyDescriptor.Builder()
+            .name("geohash-level")
+            .displayName("Geohash Level")
+            .description("The integer precision level(1-12) desired for encoding geohash")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dependsOn(MODE, ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("geohash-record-path")
+            .displayName("Geohash Record Path")
+            .description("In the encode mode, this property specifies the record path to put the geohash value;"
+                    + "in the decode mode, this property specifies the record path to retrieve the geohash value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Records that are successfully encoded or decoded will be routed to success")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Records that cannot be encoded or decoded will be routed to failure")
+            .build();
+
+    public static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            LATITUDE_RECORD_PATH,LONGITUDE_RECORD_PATH,GEOHASH_RECORD_PATH
+    ));
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        descriptors = new ArrayList<>();
+        descriptors.add(MODE);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(LATITUDE_RECORD_PATH);
+        descriptors.add(LONGITUDE_RECORD_PATH);
+        descriptors.add(GEOHASH_FORMAT);
+        descriptors.add(GEOHASH_LEVEL);
+        descriptors.add(GEOHASH_RECORD_PATH);
+        descriptors = Collections.unmodifiableList(descriptors);
+
+        relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        final RecordReaderFactory readerFactory =  context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final boolean encode = context.getProperty(MODE).getValue().equalsIgnoreCase(ENCODE_MODE);
+        final String format = context.getProperty(GEOHASH_FORMAT).getValue();
+
+        FlowFile output = session.create(input);

Review comment:
       Is there a reason for creating a new output FlowFile as opposed to changing the content of the input FlowFile? It looks like the input FlowFile is removed, so it seems like the approach could be refactored using a `StreamCallback` to process the InputStream and OuputStream together.

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";

Review comment:
       These values could be encapsulated in an `enum` named something like `ProcessingMode`.

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";
+
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+            .name("mode")
+            .displayName("Mode")
+            .description("Specifies whether to encode latitude/longitude to geohash or decode geohash to latitude/longitude")
+            .required(true)
+            .allowableValues(ENCODE_MODE, DECODE_MODE)
+            .defaultValue(ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the record reader service to use for reading incoming data")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the record writer service to use for writing data")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    public static final PropertyDescriptor LATITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("latitude-record-path")
+            .displayName("Latitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the latitude value;"
+                    + "in the decode mode, this property specifies the record path to put the latitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor LONGITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("longitude-record-path")
+            .displayName("Longitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the longitude value;"
+                    + "in the decode mode, this property specifies the record path to put the longitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_FORMAT = new PropertyDescriptor.Builder()
+            .name("geohash-format")
+            .displayName("Geohash Format")
+            .description("In the encode mode, this property specifies the desired format for encoding geohash;"
+                    + "in the decode mode, this property specifies the format of geohash provided")
+            .required(true)
+            .allowableValues(BASE32, BINARY_STRING, LONG_INT)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_LEVEL = new PropertyDescriptor.Builder()
+            .name("geohash-level")
+            .displayName("Geohash Level")
+            .description("The integer precision level(1-12) desired for encoding geohash")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dependsOn(MODE, ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("geohash-record-path")
+            .displayName("Geohash Record Path")
+            .description("In the encode mode, this property specifies the record path to put the geohash value;"
+                    + "in the decode mode, this property specifies the record path to retrieve the geohash value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Records that are successfully encoded or decoded will be routed to success")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Records that cannot be encoded or decoded will be routed to failure")
+            .build();
+
+    public static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            LATITUDE_RECORD_PATH,LONGITUDE_RECORD_PATH,GEOHASH_RECORD_PATH
+    ));
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        descriptors = new ArrayList<>();
+        descriptors.add(MODE);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(LATITUDE_RECORD_PATH);
+        descriptors.add(LONGITUDE_RECORD_PATH);
+        descriptors.add(GEOHASH_FORMAT);
+        descriptors.add(GEOHASH_LEVEL);
+        descriptors.add(GEOHASH_RECORD_PATH);
+        descriptors = Collections.unmodifiableList(descriptors);
+
+        relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile input = session.get();
+        if ( input == null ) {

Review comment:
       Recommend adjusting the spacing to remove extra spaces:
   ```suggestion
           if (input == null) {
   ```

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/test/java/org/apache/nifi/processors/geohash/GeohashRecordTest.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.geohash;
+
+import org.apache.avro.Schema;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.avro.AvroTypeUtil;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.serialization.record.MockSchemaRegistry;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;

Review comment:
       Since this is a new test class, recommend adjusting to use JUnit 5 instead of JUnit 4. The dependency is already available as part of standard dependency declarations.

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/pom.xml
##########
@@ -0,0 +1,108 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-geohash-bundle</artifactId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-geohash-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>

Review comment:
       These dependencies are included as part of the default module declarations, so they can be removed.

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";
+
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+            .name("mode")
+            .displayName("Mode")
+            .description("Specifies whether to encode latitude/longitude to geohash or decode geohash to latitude/longitude")
+            .required(true)
+            .allowableValues(ENCODE_MODE, DECODE_MODE)
+            .defaultValue(ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the record reader service to use for reading incoming data")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the record writer service to use for writing data")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    public static final PropertyDescriptor LATITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("latitude-record-path")
+            .displayName("Latitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the latitude value;"
+                    + "in the decode mode, this property specifies the record path to put the latitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor LONGITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("longitude-record-path")
+            .displayName("Longitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the longitude value;"
+                    + "in the decode mode, this property specifies the record path to put the longitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_FORMAT = new PropertyDescriptor.Builder()
+            .name("geohash-format")
+            .displayName("Geohash Format")
+            .description("In the encode mode, this property specifies the desired format for encoding geohash;"
+                    + "in the decode mode, this property specifies the format of geohash provided")
+            .required(true)
+            .allowableValues(BASE32, BINARY_STRING, LONG_INT)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_LEVEL = new PropertyDescriptor.Builder()
+            .name("geohash-level")
+            .displayName("Geohash Level")
+            .description("The integer precision level(1-12) desired for encoding geohash")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dependsOn(MODE, ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("geohash-record-path")
+            .displayName("Geohash Record Path")
+            .description("In the encode mode, this property specifies the record path to put the geohash value;"
+                    + "in the decode mode, this property specifies the record path to retrieve the geohash value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Records that are successfully encoded or decoded will be routed to success")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Records that cannot be encoded or decoded will be routed to failure")
+            .build();
+
+    public static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            LATITUDE_RECORD_PATH,LONGITUDE_RECORD_PATH,GEOHASH_RECORD_PATH
+    ));
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        descriptors = new ArrayList<>();
+        descriptors.add(MODE);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(LATITUDE_RECORD_PATH);
+        descriptors.add(LONGITUDE_RECORD_PATH);
+        descriptors.add(GEOHASH_FORMAT);
+        descriptors.add(GEOHASH_LEVEL);
+        descriptors.add(GEOHASH_RECORD_PATH);
+        descriptors = Collections.unmodifiableList(descriptors);
+
+        relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        final RecordReaderFactory readerFactory =  context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final boolean encode = context.getProperty(MODE).getValue().equalsIgnoreCase(ENCODE_MODE);
+        final String format = context.getProperty(GEOHASH_FORMAT).getValue();
+
+        FlowFile output = session.create(input);
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output)) {
+            RecordPathCache cache = new RecordPathCache(RECORD_PATH_PROPERTIES.size() + 1);
+            Map<PropertyDescriptor, RecordPath> paths = new HashMap<>();
+
+            for (PropertyDescriptor descriptor : RECORD_PATH_PROPERTIES) {
+                String rawRecordPath = context.getProperty(descriptor).evaluateAttributeExpressions(input).getValue();
+                RecordPath compiled = cache.getCompiled(rawRecordPath);
+                paths.put(descriptor, compiled);
+            }
+
+            RecordReader reader = readerFactory.createRecordReader(input, is, getLogger());
+            RecordSchema schema = writerFactory.getSchema(input.getAttributes(), reader.getSchema());
+            RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, os, output);
+
+            Record record;
+            Relationship targetRelationship = REL_FAILURE;
+            writer.beginRecordSet();
+
+            int recordsCount = 0;
+            boolean updated = false;
+            while ((record = reader.nextRecord()) != null) {
+                if(encode) {
+                    final int level = Integer.parseInt(context.getProperty(GEOHASH_LEVEL).getValue());
+                    final String rawLatitudePath = context.getProperty(LATITUDE_RECORD_PATH).evaluateAttributeExpressions(input).getValue();
+                    RecordPath latitudePath = cache.getCompiled(rawLatitudePath);
+                    final String rawLongitudePath = context.getProperty(LONGITUDE_RECORD_PATH).evaluateAttributeExpressions(input).getValue();
+                    RecordPath longitudePath = cache.getCompiled(rawLongitudePath);
+
+                    Object encodedGeohash = getEncodedGeohash(latitudePath, longitudePath, record, format, level);
+
+                    updated = updateRecord(GEOHASH_RECORD_PATH, encodedGeohash, record, paths);
+                }else {
+                    final String rawgeohashPath = context.getProperty(GEOHASH_RECORD_PATH).evaluateAttributeExpressions(input).getValue();
+                    RecordPath geohashPath = cache.getCompiled(rawgeohashPath);
+
+                    WGS84Point decodedPoint = getDecodedPointFromGeohash(geohashPath, record, format);
+
+                    if(decodedPoint != null) {
+                        updated = updateRecord(LATITUDE_RECORD_PATH, String.valueOf(decodedPoint.getLatitude()), record, paths)
+                                &&  updateRecord(LONGITUDE_RECORD_PATH, String.valueOf(decodedPoint.getLongitude()), record, paths);
+                    }
+                }
+
+                if (updated) {
+                    targetRelationship = REL_SUCCESS;
+                }
+
+                writer.write(record);
+                recordsCount++;
+
+            }
+            writer.finishRecordSet();
+            writer.close();
+
+            is.close();
+            os.close();

Review comment:
       If the implementation were refactored using a `StreamCallback` implementation, closing the InputStream and OutputStream would happen automatically.

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";
+
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+            .name("mode")
+            .displayName("Mode")
+            .description("Specifies whether to encode latitude/longitude to geohash or decode geohash to latitude/longitude")
+            .required(true)
+            .allowableValues(ENCODE_MODE, DECODE_MODE)
+            .defaultValue(ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the record reader service to use for reading incoming data")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the record writer service to use for writing data")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    public static final PropertyDescriptor LATITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("latitude-record-path")
+            .displayName("Latitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the latitude value;"
+                    + "in the decode mode, this property specifies the record path to put the latitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor LONGITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("longitude-record-path")
+            .displayName("Longitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the longitude value;"
+                    + "in the decode mode, this property specifies the record path to put the longitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_FORMAT = new PropertyDescriptor.Builder()
+            .name("geohash-format")
+            .displayName("Geohash Format")
+            .description("In the encode mode, this property specifies the desired format for encoding geohash;"
+                    + "in the decode mode, this property specifies the format of geohash provided")
+            .required(true)
+            .allowableValues(BASE32, BINARY_STRING, LONG_INT)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_LEVEL = new PropertyDescriptor.Builder()
+            .name("geohash-level")
+            .displayName("Geohash Level")
+            .description("The integer precision level(1-12) desired for encoding geohash")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dependsOn(MODE, ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("geohash-record-path")
+            .displayName("Geohash Record Path")
+            .description("In the encode mode, this property specifies the record path to put the geohash value;"
+                    + "in the decode mode, this property specifies the record path to retrieve the geohash value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Records that are successfully encoded or decoded will be routed to success")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Records that cannot be encoded or decoded will be routed to failure")
+            .build();
+
+    public static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            LATITUDE_RECORD_PATH,LONGITUDE_RECORD_PATH,GEOHASH_RECORD_PATH
+    ));
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        descriptors = new ArrayList<>();
+        descriptors.add(MODE);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(LATITUDE_RECORD_PATH);
+        descriptors.add(LONGITUDE_RECORD_PATH);
+        descriptors.add(GEOHASH_FORMAT);
+        descriptors.add(GEOHASH_LEVEL);
+        descriptors.add(GEOHASH_RECORD_PATH);
+        descriptors = Collections.unmodifiableList(descriptors);
+
+        relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        final RecordReaderFactory readerFactory =  context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final boolean encode = context.getProperty(MODE).getValue().equalsIgnoreCase(ENCODE_MODE);
+        final String format = context.getProperty(GEOHASH_FORMAT).getValue();
+
+        FlowFile output = session.create(input);
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output)) {
+            RecordPathCache cache = new RecordPathCache(RECORD_PATH_PROPERTIES.size() + 1);
+            Map<PropertyDescriptor, RecordPath> paths = new HashMap<>();
+
+            for (PropertyDescriptor descriptor : RECORD_PATH_PROPERTIES) {
+                String rawRecordPath = context.getProperty(descriptor).evaluateAttributeExpressions(input).getValue();
+                RecordPath compiled = cache.getCompiled(rawRecordPath);
+                paths.put(descriptor, compiled);
+            }
+
+            RecordReader reader = readerFactory.createRecordReader(input, is, getLogger());
+            RecordSchema schema = writerFactory.getSchema(input.getAttributes(), reader.getSchema());
+            RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, os, output);
+
+            Record record;
+            Relationship targetRelationship = REL_FAILURE;
+            writer.beginRecordSet();
+
+            int recordsCount = 0;
+            boolean updated = false;
+            while ((record = reader.nextRecord()) != null) {
+                if(encode) {
+                    final int level = Integer.parseInt(context.getProperty(GEOHASH_LEVEL).getValue());
+                    final String rawLatitudePath = context.getProperty(LATITUDE_RECORD_PATH).evaluateAttributeExpressions(input).getValue();
+                    RecordPath latitudePath = cache.getCompiled(rawLatitudePath);
+                    final String rawLongitudePath = context.getProperty(LONGITUDE_RECORD_PATH).evaluateAttributeExpressions(input).getValue();
+                    RecordPath longitudePath = cache.getCompiled(rawLongitudePath);
+
+                    Object encodedGeohash = getEncodedGeohash(latitudePath, longitudePath, record, format, level);
+
+                    updated = updateRecord(GEOHASH_RECORD_PATH, encodedGeohash, record, paths);
+                }else {

Review comment:
       Recommend running automatic formatting to correct the spacing here and on some other lines.
   ```suggestion
                   } else {
   ```

##########
File path: nifi-nar-bundles/nifi-geohash-bundle/nifi-geohash-processors/src/main/java/org/apache/nifi/processors/geohash/GeohashRecord.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.geohash;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+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.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import ch.hsr.geohash.GeoHash;
+import ch.hsr.geohash.WGS84Point;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Optional;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"geo", "geohash", "record"})
+@CapabilityDescription("A record-based processor that encodes and decodes Geohashes from and to latitude/longitude coordinates.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class GeohashRecord extends AbstractProcessor {
+    public static final String ENCODE_MODE = "Encode";
+    public static final String DECODE_MODE = "Decode";
+
+    public static final String BINARY_STRING = "BinaryString";
+    public static final String BASE32 = "Base32";
+    public static final String LONG_INT = "LongInt";
+
+    public static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
+            .name("mode")
+            .displayName("Mode")
+            .description("Specifies whether to encode latitude/longitude to geohash or decode geohash to latitude/longitude")
+            .required(true)
+            .allowableValues(ENCODE_MODE, DECODE_MODE)
+            .defaultValue(ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the record reader service to use for reading incoming data")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the record writer service to use for writing data")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    public static final PropertyDescriptor LATITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("latitude-record-path")
+            .displayName("Latitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the latitude value;"
+                    + "in the decode mode, this property specifies the record path to put the latitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor LONGITUDE_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("longitude-record-path")
+            .displayName("Longitude Record Path")
+            .description("In the encode mode, this property specifies the record path to retrieve the longitude value;"
+                    + "in the decode mode, this property specifies the record path to put the longitude value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_FORMAT = new PropertyDescriptor.Builder()
+            .name("geohash-format")
+            .displayName("Geohash Format")
+            .description("In the encode mode, this property specifies the desired format for encoding geohash;"
+                    + "in the decode mode, this property specifies the format of geohash provided")
+            .required(true)
+            .allowableValues(BASE32, BINARY_STRING, LONG_INT)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_LEVEL = new PropertyDescriptor.Builder()
+            .name("geohash-level")
+            .displayName("Geohash Level")
+            .description("The integer precision level(1-12) desired for encoding geohash")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dependsOn(MODE, ENCODE_MODE)
+            .build();
+
+    public static final PropertyDescriptor GEOHASH_RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("geohash-record-path")
+            .displayName("Geohash Record Path")
+            .description("In the encode mode, this property specifies the record path to put the geohash value;"
+                    + "in the decode mode, this property specifies the record path to retrieve the geohash value")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Records that are successfully encoded or decoded will be routed to success")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Records that cannot be encoded or decoded will be routed to failure")
+            .build();
+
+    public static final List<PropertyDescriptor> RECORD_PATH_PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            LATITUDE_RECORD_PATH,LONGITUDE_RECORD_PATH,GEOHASH_RECORD_PATH
+    ));
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        descriptors = new ArrayList<>();
+        descriptors.add(MODE);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(LATITUDE_RECORD_PATH);
+        descriptors.add(LONGITUDE_RECORD_PATH);
+        descriptors.add(GEOHASH_FORMAT);
+        descriptors.add(GEOHASH_LEVEL);
+        descriptors.add(GEOHASH_RECORD_PATH);
+        descriptors = Collections.unmodifiableList(descriptors);
+
+        relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        final RecordReaderFactory readerFactory =  context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final boolean encode = context.getProperty(MODE).getValue().equalsIgnoreCase(ENCODE_MODE);
+        final String format = context.getProperty(GEOHASH_FORMAT).getValue();
+
+        FlowFile output = session.create(input);
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output)) {
+            RecordPathCache cache = new RecordPathCache(RECORD_PATH_PROPERTIES.size() + 1);
+            Map<PropertyDescriptor, RecordPath> paths = new HashMap<>();
+
+            for (PropertyDescriptor descriptor : RECORD_PATH_PROPERTIES) {
+                String rawRecordPath = context.getProperty(descriptor).evaluateAttributeExpressions(input).getValue();
+                RecordPath compiled = cache.getCompiled(rawRecordPath);
+                paths.put(descriptor, compiled);
+            }
+
+            RecordReader reader = readerFactory.createRecordReader(input, is, getLogger());
+            RecordSchema schema = writerFactory.getSchema(input.getAttributes(), reader.getSchema());
+            RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, os, output);
+
+            Record record;
+            Relationship targetRelationship = REL_FAILURE;
+            writer.beginRecordSet();
+
+            int recordsCount = 0;
+            boolean updated = false;
+            while ((record = reader.nextRecord()) != null) {
+                if(encode) {
+                    final int level = Integer.parseInt(context.getProperty(GEOHASH_LEVEL).getValue());

Review comment:
       This can be changed to using `PropertyValue::asInteger()`:
   ```suggestion
                       final int level = context.getProperty(GEOHASH_LEVEL).asInteger();
   ```




-- 
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