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


##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/property/OutputStrategy.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.aws.kinesis.property;
+
+import org.apache.nifi.components.DescribedValue;
+
+/**
+ * Enumeration of supported Kinesis Output Strategies
+ */
+public enum OutputStrategy implements DescribedValue {
+    USE_VALUE("USE_VALUE", "Use Content as Value", "Write only the Kinesis 
Record value to the FlowFile record."),
+    USE_WRAPPER("USE_WRAPPER", "Use Wrapper", "Write the Kinesis Record value 
and metadata into the FlowFile record.");
+
+    private final String value;
+    private final String displayName;
+    private final String description;
+
+    OutputStrategy(final String value, final String displayName, final String 
description) {
+        this.value = value;
+        this.displayName = displayName;
+        this.description = description;
+    }
+
+    @Override
+    public String getValue() {
+        return value;

Review Comment:
   After removing the `value` property, this can be changed to `name()`:
   ```suggestion
           return name();
   ```



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/converter/RecordConverterIdentity.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.aws.kinesis.stream.record.converter;
+
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+public class RecordConverterIdentity implements RecordConverter {
+    @Override
+    public org.apache.nifi.serialization.record.Record convert(
+            final org.apache.nifi.serialization.record.Record record, final 
KinesisClientRecord kinesisRecord,

Review Comment:
   See note on the interface about the qualified class versus importing.



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java:
##########
@@ -117,6 +118,7 @@ public void testInvalidProperties() {
                         "'%s' validated against 'not-a-long' is invalid 
because Must be of format <duration> <TimeUnit> where <duration> is a 
non-negative integer and " +
                         "TimeUnit is a supported Time Unit, such as: nanos, 
millis, secs, mins, hrs, days\n" +
                         "'%s' validated against 'not-a-boolean' is invalid 
because Given value not found in allowed set 'true, false'\n" +
+                        "'Output Strategy' validated against 
'not-an-enum-match' is invalid because Given value not found in allowed set 
'USE_VALUE, USE_WRAPPER'\n" +

Review Comment:
   I recommend removing this test case since evaluating error messages is 
generally not useful.



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/converter/RecordConverterWrapper.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.aws.kinesis.stream.record.converter;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.Tuple;
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RecordConverterWrapper implements RecordConverter {
+
+    private static final String VALUE = "value";
+    private static final String METADATA = "metadata";
+
+    private static final String STREAM = "kinesis.stream";
+    private static final String SHARD_ID = "aws.kinesis.shard.id";
+    private static final String SEQUENCE_NUMBER = 
"aws.kinesis.sequence.number";
+    private static final String PARTITION_KEY = "aws.kinesis.partition.key";
+    private static final String APPROX_ARRIVAL_TIMESTAMP = 
"aws.kinesis.approximate.arrival.timestamp";
+
+    private static final RecordField FIELD_STREAM = new RecordField(STREAM, 
RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_SHARD_ID = new 
RecordField(SHARD_ID, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_SEQUENCE_NUMBER = new 
RecordField(SEQUENCE_NUMBER, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_PARTITION_KEY = new 
RecordField(PARTITION_KEY, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_APPROX_ARRIVAL_TIMESTAMP = new 
RecordField(APPROX_ARRIVAL_TIMESTAMP, RecordFieldType.TIMESTAMP.getDataType());
+    private static final RecordSchema SCHEMA_METADATA = new 
SimpleRecordSchema(Arrays.asList(
+            FIELD_STREAM, FIELD_SHARD_ID, FIELD_SEQUENCE_NUMBER, 
FIELD_PARTITION_KEY, FIELD_APPROX_ARRIVAL_TIMESTAMP));
+
+    public static final RecordField FIELD_METADATA = new RecordField(METADATA, 
RecordFieldType.RECORD.getRecordDataType(SCHEMA_METADATA));
+
+
+    @Override
+    public org.apache.nifi.serialization.record.Record convert(
+            final org.apache.nifi.serialization.record.Record record,
+            final KinesisClientRecord kinesisRecord, final String streamName, 
final String shardId) {
+        final Tuple<String, Object> metadata = 
toWrapperRecordMetadata(kinesisRecord, streamName, shardId);
+        return new MapRecord(convertToWriteSchema(record.getSchema()), 
Map.of(metadata.getKey(), metadata.getValue(), VALUE, record));
+    }
+
+    private Tuple<String, Object> toWrapperRecordMetadata(final 
KinesisClientRecord consumerRecord, final String streamName, final String 
shardId) {
+        final Map<String, Object> metadata = new HashMap<>();
+        metadata.put(STREAM, streamName);
+        metadata.put(SHARD_ID, shardId);
+        metadata.put(SEQUENCE_NUMBER, consumerRecord.sequenceNumber());
+        metadata.put(PARTITION_KEY, consumerRecord.partitionKey());
+        final Instant approxArrivalTimestamp = 
consumerRecord.approximateArrivalTimestamp();
+        metadata.put(APPROX_ARRIVAL_TIMESTAMP, approxArrivalTimestamp != null 
? approxArrivalTimestamp.toEpochMilli() : null);

Review Comment:
   Recommend reversing the ternary conditional as follows:
   ```suggestion
           metadata.put(APPROX_ARRIVAL_TIMESTAMP, approxArrivalTimestamp == 
null ? null : approxArrivalTimestamp.toEpochMilli());
   ```



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java:
##########
@@ -688,11 +700,15 @@ synchronized Scheduler prepareScheduler(final 
ProcessContext context, final Proc
     private ShardRecordProcessorFactory prepareRecordProcessorFactory(final 
ProcessContext context, final ProcessSessionFactory sessionFactory) {
         return () -> {
             if (isRecordReaderSet && isRecordWriterSet) {
+                final String value = 
context.getProperty(OUTPUT_STRATEGY).getValue();

Review Comment:
   This should be changed to use `asAllowableValue(OutputStrategy.class)` so 
that the comparison against the `enum` can be used.



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/converter/RecordConverterWrapper.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.aws.kinesis.stream.record.converter;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.Tuple;
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RecordConverterWrapper implements RecordConverter {
+
+    private static final String VALUE = "value";
+    private static final String METADATA = "metadata";
+
+    private static final String STREAM = "kinesis.stream";
+    private static final String SHARD_ID = "aws.kinesis.shard.id";
+    private static final String SEQUENCE_NUMBER = 
"aws.kinesis.sequence.number";
+    private static final String PARTITION_KEY = "aws.kinesis.partition.key";
+    private static final String APPROX_ARRIVAL_TIMESTAMP = 
"aws.kinesis.approximate.arrival.timestamp";
+
+    private static final RecordField FIELD_STREAM = new RecordField(STREAM, 
RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_SHARD_ID = new 
RecordField(SHARD_ID, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_SEQUENCE_NUMBER = new 
RecordField(SEQUENCE_NUMBER, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_PARTITION_KEY = new 
RecordField(PARTITION_KEY, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_APPROX_ARRIVAL_TIMESTAMP = new 
RecordField(APPROX_ARRIVAL_TIMESTAMP, RecordFieldType.TIMESTAMP.getDataType());
+    private static final RecordSchema SCHEMA_METADATA = new 
SimpleRecordSchema(Arrays.asList(
+            FIELD_STREAM, FIELD_SHARD_ID, FIELD_SEQUENCE_NUMBER, 
FIELD_PARTITION_KEY, FIELD_APPROX_ARRIVAL_TIMESTAMP));
+
+    public static final RecordField FIELD_METADATA = new RecordField(METADATA, 
RecordFieldType.RECORD.getRecordDataType(SCHEMA_METADATA));
+
+
+    @Override
+    public org.apache.nifi.serialization.record.Record convert(
+            final org.apache.nifi.serialization.record.Record record,
+            final KinesisClientRecord kinesisRecord, final String streamName, 
final String shardId) {
+        final Tuple<String, Object> metadata = 
toWrapperRecordMetadata(kinesisRecord, streamName, shardId);
+        return new MapRecord(convertToWriteSchema(record.getSchema()), 
Map.of(metadata.getKey(), metadata.getValue(), VALUE, record));
+    }
+
+    private Tuple<String, Object> toWrapperRecordMetadata(final 
KinesisClientRecord consumerRecord, final String streamName, final String 
shardId) {
+        final Map<String, Object> metadata = new HashMap<>();

Review Comment:
   Recommend using `LinkedHashMap` in this case to ensure consistent ordering.



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/property/OutputStrategy.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.aws.kinesis.property;
+
+import org.apache.nifi.components.DescribedValue;
+
+/**
+ * Enumeration of supported Kinesis Output Strategies
+ */
+public enum OutputStrategy implements DescribedValue {
+    USE_VALUE("USE_VALUE", "Use Content as Value", "Write only the Kinesis 
Record value to the FlowFile record."),
+    USE_WRAPPER("USE_WRAPPER", "Use Wrapper", "Write the Kinesis Record value 
and metadata into the FlowFile record.");

Review Comment:
   Repeating string value is not necessary:
   ```suggestion
       USE_VALUE("Use Content as Value", "Write only the Kinesis Record value 
to the FlowFile record."),
       USE_WRAPPER("Use Wrapper", "Write the Kinesis Record value and metadata 
into the FlowFile record.");
   ```



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/converter/RecordConverterWrapper.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.aws.kinesis.stream.record.converter;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.Tuple;
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RecordConverterWrapper implements RecordConverter {
+
+    private static final String VALUE = "value";
+    private static final String METADATA = "metadata";
+
+    private static final String STREAM = "kinesis.stream";

Review Comment:
   Is there a reason this property name is not prefixed with `aws` like the 
others?



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/converter/RecordConverterWrapper.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.aws.kinesis.stream.record.converter;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.util.Tuple;
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RecordConverterWrapper implements RecordConverter {
+
+    private static final String VALUE = "value";
+    private static final String METADATA = "metadata";
+
+    private static final String STREAM = "kinesis.stream";
+    private static final String SHARD_ID = "aws.kinesis.shard.id";
+    private static final String SEQUENCE_NUMBER = 
"aws.kinesis.sequence.number";
+    private static final String PARTITION_KEY = "aws.kinesis.partition.key";
+    private static final String APPROX_ARRIVAL_TIMESTAMP = 
"aws.kinesis.approximate.arrival.timestamp";
+
+    private static final RecordField FIELD_STREAM = new RecordField(STREAM, 
RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_SHARD_ID = new 
RecordField(SHARD_ID, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_SEQUENCE_NUMBER = new 
RecordField(SEQUENCE_NUMBER, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_PARTITION_KEY = new 
RecordField(PARTITION_KEY, RecordFieldType.STRING.getDataType());
+    private static final RecordField FIELD_APPROX_ARRIVAL_TIMESTAMP = new 
RecordField(APPROX_ARRIVAL_TIMESTAMP, RecordFieldType.TIMESTAMP.getDataType());
+    private static final RecordSchema SCHEMA_METADATA = new 
SimpleRecordSchema(Arrays.asList(
+            FIELD_STREAM, FIELD_SHARD_ID, FIELD_SEQUENCE_NUMBER, 
FIELD_PARTITION_KEY, FIELD_APPROX_ARRIVAL_TIMESTAMP));
+
+    public static final RecordField FIELD_METADATA = new RecordField(METADATA, 
RecordFieldType.RECORD.getRecordDataType(SCHEMA_METADATA));
+
+
+    @Override
+    public org.apache.nifi.serialization.record.Record convert(
+            final org.apache.nifi.serialization.record.Record record,
+            final KinesisClientRecord kinesisRecord, final String streamName, 
final String shardId) {
+        final Tuple<String, Object> metadata = 
toWrapperRecordMetadata(kinesisRecord, streamName, shardId);
+        return new MapRecord(convertToWriteSchema(record.getSchema()), 
Map.of(metadata.getKey(), metadata.getValue(), VALUE, record));
+    }
+
+    private Tuple<String, Object> toWrapperRecordMetadata(final 
KinesisClientRecord consumerRecord, final String streamName, final String 
shardId) {
+        final Map<String, Object> metadata = new HashMap<>();
+        metadata.put(STREAM, streamName);
+        metadata.put(SHARD_ID, shardId);
+        metadata.put(SEQUENCE_NUMBER, consumerRecord.sequenceNumber());
+        metadata.put(PARTITION_KEY, consumerRecord.partitionKey());
+        final Instant approxArrivalTimestamp = 
consumerRecord.approximateArrivalTimestamp();
+        metadata.put(APPROX_ARRIVAL_TIMESTAMP, approxArrivalTimestamp != null 
? approxArrivalTimestamp.toEpochMilli() : null);
+        final org.apache.nifi.serialization.record.Record record = new 
MapRecord(SCHEMA_METADATA, metadata);
+        return new Tuple<>(METADATA, record);
+    }
+
+    private RecordSchema convertToWriteSchema(final RecordSchema readerSchema) 
{
+        final RecordField recordField = new RecordField(VALUE, 
RecordFieldType.RECORD.getRecordDataType(readerSchema));
+        return new SimpleRecordSchema(Arrays.asList(FIELD_METADATA, 
recordField));

Review Comment:
   `List.of()` can be used here.



##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/converter/RecordConverter.java:
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.aws.kinesis.stream.record.converter;
+
+import software.amazon.kinesis.retrieval.KinesisClientRecord;
+
+public interface RecordConverter {
+
+    org.apache.nifi.serialization.record.Record convert(
+            final org.apache.nifi.serialization.record.Record record, final 
KinesisClientRecord kinesisRecord,

Review Comment:
   It looks like the fully-qualified class can be replaced with a standard 
import, will that work?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to