You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/12/06 17:16:39 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6416: NIFI-10234 implement PutIoTDB

exceptionfactory commented on code in PR #6416:
URL: https://github.com/apache/nifi/pull/6416#discussion_r1041235006


##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;
+
+            Record record;
+
+            while ((record = recordReader.nextRecord()) != null) {
+                long timestamp = getTimestamp(schema, record);
+                boolean isFulled = false;
+
+                for (Map.Entry<String, Tablet> entry : tablets.entrySet()) {
+                    String device = entry.getKey();
+                    Tablet tablet = entry.getValue();
+                    int rowIndex = tablet.rowSize++;
+
+                    tablet.addTimestamp(rowIndex, timestamp);
+                    List<MeasurementSchema> measurements = tablet.getSchemas();
+                    for (MeasurementSchema measurement : measurements) {
+                        String id = measurement.getMeasurementId();
+                        TSDataType type = measurement.getType();
+                        Object value = getValue(record.getValue(id), type);
+                        tablet.addValue(id, rowIndex, value);
+                    }
+                    isFulled = tablet.rowSize == tablet.getMaxRowNumber();
+                }
+                if (isFulled) {
+                    if (aligned) {
+                        session.get().insertAlignedTablets(tablets);
+                    } else {
+                        session.get().insertTablets(tablets);
+                    }
+                    tablets.values().forEach(tablet -> tablet.reset());
+                }
+            }
+
+            AtomicBoolean hasRest = new AtomicBoolean(false);
+            tablets.forEach(
+                    (device, tablet) -> {
+                        if (hasRest.get() == false && tablet.rowSize != 0) {
+                            hasRest.set(true);
+                        }
+                    });
+            if (hasRest.get()) {
+                if (aligned) {
+                    session.get().insertAlignedTablets(tablets);
+                } else {
+                    session.get().insertTablets(tablets);
+                }
+            }
+        } catch (Exception e) {
+            getLogger().error("Processing failed {}", flowFile, e);
+            processSession.transfer(flowFile, REL_FAILURE);
+            e.printStackTrace();

Review Comment:
   `e.printStackTrace()` should be removed.



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;
+
+            Record record;
+
+            while ((record = recordReader.nextRecord()) != null) {
+                long timestamp = getTimestamp(schema, record);
+                boolean isFulled = false;
+
+                for (Map.Entry<String, Tablet> entry : tablets.entrySet()) {
+                    String device = entry.getKey();
+                    Tablet tablet = entry.getValue();
+                    int rowIndex = tablet.rowSize++;
+
+                    tablet.addTimestamp(rowIndex, timestamp);
+                    List<MeasurementSchema> measurements = tablet.getSchemas();
+                    for (MeasurementSchema measurement : measurements) {
+                        String id = measurement.getMeasurementId();
+                        TSDataType type = measurement.getType();
+                        Object value = getValue(record.getValue(id), type);
+                        tablet.addValue(id, rowIndex, value);
+                    }
+                    isFulled = tablet.rowSize == tablet.getMaxRowNumber();

Review Comment:
   ```suggestion
                       filled = tablet.rowSize == tablet.getMaxRowNumber();
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;
+
+            Record record;
+
+            while ((record = recordReader.nextRecord()) != null) {
+                long timestamp = getTimestamp(schema, record);
+                boolean isFulled = false;
+
+                for (Map.Entry<String, Tablet> entry : tablets.entrySet()) {
+                    String device = entry.getKey();
+                    Tablet tablet = entry.getValue();
+                    int rowIndex = tablet.rowSize++;
+
+                    tablet.addTimestamp(rowIndex, timestamp);
+                    List<MeasurementSchema> measurements = tablet.getSchemas();
+                    for (MeasurementSchema measurement : measurements) {
+                        String id = measurement.getMeasurementId();
+                        TSDataType type = measurement.getType();
+                        Object value = getValue(record.getValue(id), type);
+                        tablet.addValue(id, rowIndex, value);
+                    }
+                    isFulled = tablet.rowSize == tablet.getMaxRowNumber();
+                }
+                if (isFulled) {
+                    if (aligned) {
+                        session.get().insertAlignedTablets(tablets);
+                    } else {
+                        session.get().insertTablets(tablets);
+                    }
+                    tablets.values().forEach(tablet -> tablet.reset());
+                }
+            }
+
+            AtomicBoolean hasRest = new AtomicBoolean(false);
+            tablets.forEach(
+                    (device, tablet) -> {
+                        if (hasRest.get() == false && tablet.rowSize != 0) {
+                            hasRest.set(true);
+                        }
+                    });
+            if (hasRest.get()) {
+                if (aligned) {
+                    session.get().insertAlignedTablets(tablets);
+                } else {
+                    session.get().insertTablets(tablets);
+                }
+            }
+        } catch (Exception e) {
+            getLogger().error("Processing failed {}", flowFile, e);
+            processSession.transfer(flowFile, REL_FAILURE);
+            e.printStackTrace();
+            return;
+        }
+        processSession.transfer(flowFile, REL_SUCCESS);
+    }
+
+    private IoTDBSchema getSchema(String property, RecordReader recordReader) throws Exception {
+        ValidationResult result;
+        IoTDBSchema schema;
+        result =
+                property != null
+                        ? validateSchemaAttribute(property)
+                        : validateSchema(recordReader.getSchema());
+
+        if (!result.getKey()) {
+            getLogger().error("The property `schema` has an error: {}", result.getValue());
+            throw new Exception();
+
+        } else {
+            if (result.getValue() != null) {
+                getLogger().warn("The property `schema` has a warn: {}", result.getValue());
+            }
+        }
+
+        schema =
+                property != null
+                        ? mapper.readValue(property, IoTDBSchema.class)
+                        : convertSchema(recordReader.getSchema());
+        return schema;
+    }
+
+    private long getTimestamp(IoTDBSchema schema, Record record) {
+        long timestamp;
+        Object time = record.getValue(schema.getTimeName());
+        if (time instanceof  Timestamp) {
+            Timestamp temp = (Timestamp) time;
+            timestamp = temp.getTime();
+        } else if (time instanceof  Time) {
+            Time temp = (Time) time;
+            timestamp = temp.getTime();
+        } else if (time instanceof  Date) {
+            Date temp = (Date) time;
+            timestamp = temp.getTime();
+        } else
+            timestamp = (Long) time;
+        return timestamp;
+    }
+
+    private Object getValue(Object value, TSDataType type) throws Exception {
+        if (value != null) {
+            try {
+                value = convertType(value, type);
+            } catch (Exception e) {
+                throw new Exception(String.format("The value {%s} can't be converted to the type {%s}",

Review Comment:
   This should be changed to an `IllegalArgumentException`:
   ```suggestion
                   throw new IllegalArgumentException(String.format("The value [%s] cannot be converted to the type [%s]",
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;
+
+            Record record;
+
+            while ((record = recordReader.nextRecord()) != null) {
+                long timestamp = getTimestamp(schema, record);
+                boolean isFulled = false;
+
+                for (Map.Entry<String, Tablet> entry : tablets.entrySet()) {
+                    String device = entry.getKey();
+                    Tablet tablet = entry.getValue();
+                    int rowIndex = tablet.rowSize++;
+
+                    tablet.addTimestamp(rowIndex, timestamp);
+                    List<MeasurementSchema> measurements = tablet.getSchemas();
+                    for (MeasurementSchema measurement : measurements) {
+                        String id = measurement.getMeasurementId();
+                        TSDataType type = measurement.getType();
+                        Object value = getValue(record.getValue(id), type);
+                        tablet.addValue(id, rowIndex, value);
+                    }
+                    isFulled = tablet.rowSize == tablet.getMaxRowNumber();
+                }
+                if (isFulled) {
+                    if (aligned) {
+                        session.get().insertAlignedTablets(tablets);
+                    } else {
+                        session.get().insertTablets(tablets);
+                    }
+                    tablets.values().forEach(tablet -> tablet.reset());
+                }
+            }
+
+            AtomicBoolean hasRest = new AtomicBoolean(false);
+            tablets.forEach(
+                    (device, tablet) -> {
+                        if (hasRest.get() == false && tablet.rowSize != 0) {
+                            hasRest.set(true);
+                        }
+                    });
+            if (hasRest.get()) {
+                if (aligned) {
+                    session.get().insertAlignedTablets(tablets);
+                } else {
+                    session.get().insertTablets(tablets);
+                }
+            }
+        } catch (Exception e) {
+            getLogger().error("Processing failed {}", flowFile, e);
+            processSession.transfer(flowFile, REL_FAILURE);
+            e.printStackTrace();
+            return;
+        }
+        processSession.transfer(flowFile, REL_SUCCESS);
+    }
+
+    private IoTDBSchema getSchema(String property, RecordReader recordReader) throws Exception {
+        ValidationResult result;
+        IoTDBSchema schema;
+        result =
+                property != null
+                        ? validateSchemaAttribute(property)
+                        : validateSchema(recordReader.getSchema());
+
+        if (!result.getKey()) {
+            getLogger().error("The property `schema` has an error: {}", result.getValue());
+            throw new Exception();
+
+        } else {
+            if (result.getValue() != null) {
+                getLogger().warn("The property `schema` has a warn: {}", result.getValue());
+            }
+        }
+
+        schema =
+                property != null
+                        ? mapper.readValue(property, IoTDBSchema.class)
+                        : convertSchema(recordReader.getSchema());
+        return schema;
+    }
+
+    private long getTimestamp(IoTDBSchema schema, Record record) {
+        long timestamp;
+        Object time = record.getValue(schema.getTimeName());
+        if (time instanceof  Timestamp) {
+            Timestamp temp = (Timestamp) time;
+            timestamp = temp.getTime();
+        } else if (time instanceof  Time) {
+            Time temp = (Time) time;
+            timestamp = temp.getTime();
+        } else if (time instanceof  Date) {
+            Date temp = (Date) time;
+            timestamp = temp.getTime();
+        } else
+            timestamp = (Long) time;
+        return timestamp;

Review Comment:
   It seems unsafe to default to `Long` without checking the type.
   ```suggestion
           } else if (time intanceof Long) {
               timestamp = (Long) time;
           } else {
               throw new IllegalArgumentException(String.format("Unexpected Time Field: %s", time));
           }
           return timestamp;
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;
+
+            Record record;
+
+            while ((record = recordReader.nextRecord()) != null) {
+                long timestamp = getTimestamp(schema, record);
+                boolean isFulled = false;
+
+                for (Map.Entry<String, Tablet> entry : tablets.entrySet()) {
+                    String device = entry.getKey();
+                    Tablet tablet = entry.getValue();
+                    int rowIndex = tablet.rowSize++;
+
+                    tablet.addTimestamp(rowIndex, timestamp);
+                    List<MeasurementSchema> measurements = tablet.getSchemas();
+                    for (MeasurementSchema measurement : measurements) {
+                        String id = measurement.getMeasurementId();
+                        TSDataType type = measurement.getType();
+                        Object value = getValue(record.getValue(id), type);
+                        tablet.addValue(id, rowIndex, value);
+                    }
+                    isFulled = tablet.rowSize == tablet.getMaxRowNumber();
+                }
+                if (isFulled) {
+                    if (aligned) {
+                        session.get().insertAlignedTablets(tablets);
+                    } else {
+                        session.get().insertTablets(tablets);
+                    }
+                    tablets.values().forEach(tablet -> tablet.reset());
+                }
+            }
+
+            AtomicBoolean hasRest = new AtomicBoolean(false);
+            tablets.forEach(
+                    (device, tablet) -> {
+                        if (hasRest.get() == false && tablet.rowSize != 0) {
+                            hasRest.set(true);
+                        }
+                    });
+            if (hasRest.get()) {
+                if (aligned) {
+                    session.get().insertAlignedTablets(tablets);
+                } else {
+                    session.get().insertTablets(tablets);
+                }
+            }
+        } catch (Exception e) {
+            getLogger().error("Processing failed {}", flowFile, e);
+            processSession.transfer(flowFile, REL_FAILURE);
+            e.printStackTrace();
+            return;
+        }
+        processSession.transfer(flowFile, REL_SUCCESS);
+    }
+
+    private IoTDBSchema getSchema(String property, RecordReader recordReader) throws Exception {
+        ValidationResult result;
+        IoTDBSchema schema;
+        result =
+                property != null
+                        ? validateSchemaAttribute(property)
+                        : validateSchema(recordReader.getSchema());
+
+        if (!result.getKey()) {
+            getLogger().error("The property `schema` has an error: {}", result.getValue());
+            throw new Exception();
+
+        } else {
+            if (result.getValue() != null) {
+                getLogger().warn("The property `schema` has a warn: {}", result.getValue());
+            }
+        }
+
+        schema =
+                property != null
+                        ? mapper.readValue(property, IoTDBSchema.class)
+                        : convertSchema(recordReader.getSchema());
+        return schema;
+    }
+
+    private long getTimestamp(IoTDBSchema schema, Record record) {
+        long timestamp;
+        Object time = record.getValue(schema.getTimeName());
+        if (time instanceof  Timestamp) {
+            Timestamp temp = (Timestamp) time;
+            timestamp = temp.getTime();
+        } else if (time instanceof  Time) {
+            Time temp = (Time) time;
+            timestamp = temp.getTime();
+        } else if (time instanceof  Date) {
+            Date temp = (Date) time;
+            timestamp = temp.getTime();
+        } else
+            timestamp = (Long) time;
+        return timestamp;
+    }
+
+    private Object getValue(Object value, TSDataType type) throws Exception {
+        if (value != null) {
+            try {
+                value = convertType(value, type);
+            } catch (Exception e) {
+                throw new Exception(String.format("The value {%s} can't be converted to the type {%s}",
+                        value, type));
+            }
+        } else
+            value = null;
+        return value;

Review Comment:
   Please use curly braces for all conditional blocks:
   ```suggestion
           } else {
               value = null;
           }
           return value;
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/model/ValidationResult.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.model;
+
+public class ValidationResult {
+    private Boolean result;

Review Comment:
   This should be changed to a primitive.
   ```suggestion
       private boolean result;
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;
+
+            Record record;
+
+            while ((record = recordReader.nextRecord()) != null) {
+                long timestamp = getTimestamp(schema, record);
+                boolean isFulled = false;
+
+                for (Map.Entry<String, Tablet> entry : tablets.entrySet()) {
+                    String device = entry.getKey();
+                    Tablet tablet = entry.getValue();
+                    int rowIndex = tablet.rowSize++;
+
+                    tablet.addTimestamp(rowIndex, timestamp);
+                    List<MeasurementSchema> measurements = tablet.getSchemas();
+                    for (MeasurementSchema measurement : measurements) {
+                        String id = measurement.getMeasurementId();
+                        TSDataType type = measurement.getType();
+                        Object value = getValue(record.getValue(id), type);
+                        tablet.addValue(id, rowIndex, value);
+                    }
+                    isFulled = tablet.rowSize == tablet.getMaxRowNumber();
+                }
+                if (isFulled) {
+                    if (aligned) {
+                        session.get().insertAlignedTablets(tablets);
+                    } else {
+                        session.get().insertTablets(tablets);
+                    }
+                    tablets.values().forEach(tablet -> tablet.reset());
+                }
+            }
+
+            AtomicBoolean hasRest = new AtomicBoolean(false);
+            tablets.forEach(
+                    (device, tablet) -> {
+                        if (hasRest.get() == false && tablet.rowSize != 0) {
+                            hasRest.set(true);
+                        }
+                    });
+            if (hasRest.get()) {
+                if (aligned) {
+                    session.get().insertAlignedTablets(tablets);
+                } else {
+                    session.get().insertTablets(tablets);
+                }
+            }
+        } catch (Exception e) {
+            getLogger().error("Processing failed {}", flowFile, e);
+            processSession.transfer(flowFile, REL_FAILURE);
+            e.printStackTrace();
+            return;
+        }
+        processSession.transfer(flowFile, REL_SUCCESS);
+    }
+
+    private IoTDBSchema getSchema(String property, RecordReader recordReader) throws Exception {
+        ValidationResult result;
+        IoTDBSchema schema;
+        result =
+                property != null
+                        ? validateSchemaAttribute(property)
+                        : validateSchema(recordReader.getSchema());
+
+        if (!result.getKey()) {
+            getLogger().error("The property `schema` has an error: {}", result.getValue());
+            throw new Exception();

Review Comment:
   An empty `Exception` should not be thrown. This should be a more specific type of Exception, such as IllegalStateException, and a message should always be included.



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/test/java/org/apache/nifi/processors/AbstractIoTDBUT.java:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.model.ValidationResult;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class AbstractIoTDBUT {
+    private static TestAbstractIoTDBProcessor processor;
+
+    @Before
+    public void init() {
+        processor = new TestAbstractIoTDBProcessor();
+    }
+
+    @Test
+    public void testValidateSchemaAttribute() {
+        // normal schema
+        String schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        ValidationResult result = processor.validateSchemaAttribute(schemaAttribute);
+         assertTrue(result.getKey());
+         assertEquals(null, result.getValue());
+
+        // schema with wrong field
+        schemaAttribute =
+                "{\n"
+                        + "\t\"time\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        String exceptedMsg = "The JSON of schema must contain `timeName` and `fields`";
+
+         assertEquals(false, result.getKey());
+         assertEquals(exceptedMsg, result.getValue());
+
+        // schema without tsName
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg = "`tsName` or `dataType` has not been set";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema without data type
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg = "`tsName` or `dataType` has not been set";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema with wrong data type
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg =
+                "Unknown `dataType`: INT. The supported dataTypes are [FLOAT, INT64, INT32, TEXT, DOUBLE, BOOLEAN]";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema with wrong key
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encode\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg = "Unknown property or properties: [encode]";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema with wrong compression type
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\",\n"
+                        + "\t\t\"compressionType\": \"ZIP\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\",\n"
+                        + "\t\t\"compressionType\": \"GZIP\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg =
+                "Unknown `compressionType`: ZIP, The supported compressionType are [LZO, PAA, SDT, UNCOMPRESSED, PLA, LZ4, GZIP, SNAPPY]";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+    }
+
+    @Test
+    public void testParseSchema() {
+        ArrayList<String> filedNames =
+                new ArrayList<String>() {
+                    {
+                        add("root.sg1.d1.s1");
+                        add("root.sg1.d1.s2");
+                        add("root.sg1.d2.s1");
+                    }
+                };
+        Map<String, List<String>> deviceMeasurementMap = processor.parseSchema(filedNames);
+        HashMap<String, List<String>> exceptedMap =
+                new HashMap<String, List<String>>() {
+                    {
+                        put(
+                                "root.sg1.d1",
+                                new ArrayList<String>() {
+                                    {
+                                        add("s1");
+                                        add("s2");
+                                    }
+                                });

Review Comment:
   This kind of anonymous instantiation should not be used. Instead, Arrays.asList() should be used, please update all references.



##########
nifi-nar-bundles/nifi-iotdb-bundle/pom.xml:
##########
@@ -0,0 +1,48 @@
+<?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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-nar-bundles</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.19.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>nifi-iotdb-bundle</artifactId>
+
+    <modules>
+        <module>nifi-iotdb-processors</module>
+        <module>nifi-iotdb-nar</module>
+    </modules>
+    <packaging>pom</packaging>
+    <properties>
+        <iotdb.sdk.version>0.13.0</iotdb.sdk.version>

Review Comment:
   This could now be upgraded to 1.0.0



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/test/java/org/apache/nifi/processors/PutIoTDBIT.java:
##########
@@ -0,0 +1,331 @@
+/*
+ * 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;
+
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.session.Session;
+import org.apache.iotdb.session.SessionDataSet;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;

Review Comment:
   See note on refactoring using JUnit 5 Jupiter



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);
+            processSession.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        final boolean aligned = Boolean.valueOf(alignedProperty);
+        int maxRowNumber = Integer.valueOf(maxRowNumberProperty);
+
+        try (final InputStream inputStream = processSession.read(flowFile);
+             final RecordReader recordReader =
+                     recordParserFactory.createRecordReader(flowFile, inputStream, getLogger())) {
+            IoTDBSchema schema = getSchema(schemaProperty, recordReader);
+
+            List<String> fieldNames = schema.getFieldNames(prefix);
+
+            HashMap<String, Tablet> tablets = generateTablets(schema, prefix, maxRowNumber);
+            DateTimeFormatter format = null;

Review Comment:
   It looks like this can be removed:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/model/ValidationResult.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.model;
+
+public class ValidationResult {
+    private Boolean result;
+    private String message;
+
+    public ValidationResult(Boolean result, String message) {

Review Comment:
   ```suggestion
       public ValidationResult(boolean result, String message) {
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/test/java/org/apache/nifi/processors/AbstractIoTDBUT.java:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.model.ValidationResult;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;

Review Comment:
   These imports and usage should be changed to JUnit 5 Jupiter



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/test/java/org/apache/nifi/processors/AbstractIoTDBUT.java:
##########
@@ -0,0 +1,282 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.model.ValidationResult;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class AbstractIoTDBUT {
+    private static TestAbstractIoTDBProcessor processor;
+
+    @Before
+    public void init() {
+        processor = new TestAbstractIoTDBProcessor();
+    }
+
+    @Test
+    public void testValidateSchemaAttribute() {
+        // normal schema
+        String schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        ValidationResult result = processor.validateSchemaAttribute(schemaAttribute);
+         assertTrue(result.getKey());
+         assertEquals(null, result.getValue());
+
+        // schema with wrong field
+        schemaAttribute =
+                "{\n"
+                        + "\t\"time\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        String exceptedMsg = "The JSON of schema must contain `timeName` and `fields`";
+
+         assertEquals(false, result.getKey());
+         assertEquals(exceptedMsg, result.getValue());
+
+        // schema without tsName
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg = "`tsName` or `dataType` has not been set";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema without data type
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg = "`tsName` or `dataType` has not been set";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema with wrong data type
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg =
+                "Unknown `dataType`: INT. The supported dataTypes are [FLOAT, INT64, INT32, TEXT, DOUBLE, BOOLEAN]";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema with wrong key
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encode\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg = "Unknown property or properties: [encode]";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+
+        // schema with wrong compression type
+        schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\",\n"
+                        + "\t\t\"compressionType\": \"ZIP\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\",\n"
+                        + "\t\t\"compressionType\": \"GZIP\"\n"
+                        + "\t}]\n"
+                        + "}";
+
+        result = processor.validateSchemaAttribute(schemaAttribute);
+        exceptedMsg =
+                "Unknown `compressionType`: ZIP, The supported compressionType are [LZO, PAA, SDT, UNCOMPRESSED, PLA, LZ4, GZIP, SNAPPY]";
+
+        assertEquals(false, result.getKey());
+        assertEquals(exceptedMsg, result.getValue());
+    }
+
+    @Test
+    public void testParseSchema() {
+        ArrayList<String> filedNames =
+                new ArrayList<String>() {
+                    {
+                        add("root.sg1.d1.s1");
+                        add("root.sg1.d1.s2");
+                        add("root.sg1.d2.s1");
+                    }
+                };
+        Map<String, List<String>> deviceMeasurementMap = processor.parseSchema(filedNames);
+        HashMap<String, List<String>> exceptedMap =
+                new HashMap<String, List<String>>() {
+                    {
+                        put(
+                                "root.sg1.d1",
+                                new ArrayList<String>() {
+                                    {
+                                        add("s1");
+                                        add("s2");
+                                    }
+                                });
+                        put(
+                                "root.sg1.d2",
+                                new ArrayList<String>() {
+                                    {
+                                        add("s1");
+                                    }
+                                });
+                    }
+                };
+        assertEquals(exceptedMap, deviceMeasurementMap);
+    }
+
+    @Test
+    public void testGenerateTablet() throws JsonProcessingException {
+        String schemaAttribute =
+                "{\n"
+                        + "\t\"timeName\": \"Time\",\n"
+                        + "\t\"fields\": [{\n"
+                        + "\t\t\"tsName\": \"s1\",\n"
+                        + "\t\t\"dataType\": \"INT32\",\n"
+                        + "\t\t\"encoding\": \"RLE\"\n"
+                        + "\t}, {\n"
+                        + "\t\t\"tsName\": \"s2\",\n"
+                        + "\t\t\"dataType\": \"DOUBLE\",\n"
+                        + "\t\t\"encoding\": \"PLAIN\"\n"
+                        + "\t}]\n"
+                        + "}";
+        IoTDBSchema schema = new ObjectMapper().readValue(schemaAttribute, IoTDBSchema.class);
+        HashMap<String, Tablet> tablets = processor.generateTablets(schema, "root.test_sg.test_d1." ,1);
+
+        HashMap<String, Tablet> exceptedTablets = new HashMap<>();
+        ArrayList<MeasurementSchema> schemas = new ArrayList<>();

Review Comment:
   ```suggestion
           List<MeasurementSchema> schemas = new ArrayList<>();
   ```



##########
nifi-nar-bundles/nifi-iotdb-bundle/nifi-iotdb-processors/src/main/java/org/apache/nifi/processors/PutIoTDBRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import java.io.InputStream;
+import java.time.format.DateTimeFormatter;
+import java.util.Set;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processors.model.IoTDBSchema;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+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.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.processors.model.ValidationResult;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import java.sql.Timestamp;
+import java.sql.Time;
+import java.sql.Date;
+
+@Tags({"iotdb", "insert", "tablet"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription(
+        "This is a record aware processor that reads the content of the incoming FlowFile as individual records using the "
+                + "configured 'Record Reader' and writes them to Apache IoTDB using native interface.")
+public class PutIoTDBRecord extends AbstractIoTDB {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY =
+            new PropertyDescriptor.Builder()
+                    .name("Record Reader")
+                    .description(
+                            "Specifies the type of Record Reader controller service to use for parsing the incoming data "
+                                    + "and determining the schema")
+                    .identifiesControllerService(RecordReaderFactory.class)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor SCHEMA =
+            new PropertyDescriptor.Builder()
+                    .name("Schema Template")
+                    .description(
+                            "The Apache IoTDB Schema Template defined using JSON.\n" +
+                                    "The Processor will infer the IoTDB Schema when this property is not configured.\n" +
+                                    "Besides, you can set encoding type and compression type by this method.\n" +
+                                    "If you want to know more detail about this, you can browse this link: https://iotdb.apache.org/UserGuide/Master/Ecosystem-Integration/NiFi-IoTDB.html")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .build();
+
+    static final PropertyDescriptor PREFIX =
+            new PropertyDescriptor.Builder()
+                    .name("Prefix")
+                    .description(
+                            "The Prefix begin with root. that will be add to the tsName in data.\n")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(true)
+                    .build();
+
+    static final PropertyDescriptor ALIGNED =
+            new PropertyDescriptor.Builder()
+                    .name("Aligned")
+                    .description("Whether to use the Apache IoTDB Aligned Timeseries interface")
+                    .allowableValues("true", "false")
+                    .defaultValue("false")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("false")
+                    .build();
+
+    static final PropertyDescriptor MAX_ROW_NUMBER =
+            new PropertyDescriptor.Builder()
+                    .name("Max Row Number")
+                    .description(
+                            "Specifies the max row number of each Apache IoTDB Tablet")
+                    .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                    .required(false)
+                    .defaultValue("1024")
+                    .build();
+
+    private static final String ROOTPREFIX = "root.";
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        super.init(context);
+        descriptors.add(RECORD_READER_FACTORY);
+        descriptors.add(SCHEMA);
+        descriptors.add(PREFIX);
+        descriptors.add(ALIGNED);
+        descriptors.add(MAX_ROW_NUMBER);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext processContext, ProcessSession processSession)
+            throws ProcessException {
+
+        FlowFile flowFile = processSession.get();
+
+        if (flowFile == null) {
+            return;
+        }
+
+        String schemaProperty = processContext.getProperty(SCHEMA).evaluateAttributeExpressions(flowFile).getValue();
+        String alignedProperty = processContext.getProperty(ALIGNED).evaluateAttributeExpressions(flowFile).getValue();
+        String maxRowNumberProperty = processContext.getProperty(MAX_ROW_NUMBER).evaluateAttributeExpressions(flowFile).getValue();
+        String prefix = processContext.getProperty(PREFIX).evaluateAttributeExpressions(flowFile).getValue();
+        final RecordReaderFactory recordParserFactory =
+                processContext
+                        .getProperty(RECORD_READER_FACTORY)
+                        .asControllerService(RecordReaderFactory.class);
+
+        if (!prefix.startsWith(ROOTPREFIX) || !prefix.endsWith(".")) {
+            getLogger().error("The prefix is not begin with root and end with .", flowFile);

Review Comment:
   ```suggestion
               getLogger().error("The Prefix does not begin with [root] and end with [.]", flowFile);
   ```



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