mirror of https://github.com/apache/nifi.git
NIFI-9458 Replaced SimpleDateFormat with DateTimeFormatter (#8248)
- Added FieldConverterRegistry and Standard implementation - Moved DateTypeUtils methods to FieldConverter implementations
This commit is contained in:
parent
b47ca20f56
commit
250fe90b34
|
@ -18,10 +18,8 @@ package org.apache.nifi.logging;
|
|||
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.sql.Date;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Locale;
|
||||
import java.time.Instant;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
|
||||
public class LogMessage {
|
||||
|
||||
|
@ -32,8 +30,9 @@ public class LogMessage {
|
|||
private final String flowFileUuid;
|
||||
private final Object[] objects;
|
||||
|
||||
public static final String DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
|
||||
public static final String TO_STRING_FORMAT = "%1$s %2$s - %3$s";
|
||||
private static final String DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
|
||||
private static final String TO_STRING_FORMAT = "%1$s %2$s - %3$s";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(DATE_TIME_FORMAT);
|
||||
|
||||
public static class Builder {
|
||||
|
||||
|
@ -109,8 +108,7 @@ public class LogMessage {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
final DateFormat dateFormat = new SimpleDateFormat(DATE_TIME_FORMAT, Locale.US);
|
||||
final String formattedTime = dateFormat.format(new Date(time));
|
||||
final String formattedTime = DATE_TIME_FORMATTER.format(Instant.ofEpochMilli(time));
|
||||
|
||||
String formattedMsg = String.format(TO_STRING_FORMAT, formattedTime, logLevel.toString(), message);
|
||||
if (throwable != null) {
|
||||
|
|
|
@ -21,10 +21,14 @@ import org.apache.nifi.record.path.RecordPathEvaluationContext;
|
|||
import org.apache.nifi.record.path.StandardFieldValue;
|
||||
import org.apache.nifi.record.path.paths.RecordPathSegment;
|
||||
import org.apache.nifi.record.path.util.RecordPathUtils;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class Format extends RecordPathSegment {
|
||||
|
@ -52,22 +56,37 @@ public class Format extends RecordPathSegment {
|
|||
final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
|
||||
return fieldValues.filter(fv -> fv.getValue() != null)
|
||||
.map(fv -> {
|
||||
final java.text.DateFormat dateFormat = getDateFormat(this.dateFormat, this.timeZoneID, context);
|
||||
if (dateFormat == null) {
|
||||
final DateTimeFormatter dateTimeFormatter = getDateTimeFormatter(this.dateFormat, this.timeZoneID, context);
|
||||
if (dateTimeFormatter == null) {
|
||||
return fv;
|
||||
}
|
||||
|
||||
if (!(fv.getValue() instanceof Date) && !(fv.getValue() instanceof Number)) {
|
||||
final Object fieldValue = fv.getValue();
|
||||
|
||||
final Instant instant;
|
||||
if (fieldValue instanceof Date dateField) {
|
||||
instant = Instant.ofEpochMilli(dateField.getTime());
|
||||
} else if (fieldValue instanceof Number numberField) {
|
||||
instant = Instant.ofEpochMilli(numberField.longValue());
|
||||
} else {
|
||||
return fv;
|
||||
}
|
||||
|
||||
final Date dateValue = DataTypeUtils.toDate(fv.getValue(), null, fv.getField().getFieldName());
|
||||
final String formatted = dateFormat.format(dateValue);
|
||||
final ZoneId zoneId;
|
||||
if (timeZoneID == null) {
|
||||
zoneId = ZoneId.systemDefault();
|
||||
} else {
|
||||
final String timeZoneId = RecordPathUtils.getFirstStringValue(timeZoneID, context);
|
||||
zoneId = TimeZone.getTimeZone(timeZoneId).toZoneId();
|
||||
}
|
||||
|
||||
final ZonedDateTime dateTime = instant.atZone(zoneId);
|
||||
final String formatted = dateTimeFormatter.format(dateTime);
|
||||
return new StandardFieldValue(formatted, fv.getField(), fv.getParent().orElse(null));
|
||||
});
|
||||
}
|
||||
|
||||
private java.text.DateFormat getDateFormat(final RecordPathSegment dateFormatSegment, final RecordPathSegment timeZoneID, final RecordPathEvaluationContext context) {
|
||||
private DateTimeFormatter getDateTimeFormatter(final RecordPathSegment dateFormatSegment, final RecordPathSegment timeZoneID, final RecordPathEvaluationContext context) {
|
||||
final String dateFormatString = RecordPathUtils.getFirstStringValue(dateFormatSegment, context);
|
||||
if (StringUtils.isEmpty(dateFormatString)) {
|
||||
return null;
|
||||
|
@ -75,13 +94,14 @@ public class Format extends RecordPathSegment {
|
|||
|
||||
try {
|
||||
if (timeZoneID == null) {
|
||||
return DataTypeUtils.getDateFormat(dateFormatString);
|
||||
return DateTimeFormatter.ofPattern(dateFormatString);
|
||||
} else {
|
||||
final String timeZoneStr = RecordPathUtils.getFirstStringValue(timeZoneID, context);
|
||||
if (StringUtils.isEmpty(timeZoneStr)) {
|
||||
return null;
|
||||
}
|
||||
return DataTypeUtils.getDateFormat(dateFormatString, timeZoneStr);
|
||||
final ZoneId zoneId = TimeZone.getTimeZone(timeZoneStr).toZoneId();
|
||||
return DateTimeFormatter.ofPattern(dateFormatString).withZone(zoneId);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
return null;
|
||||
|
|
|
@ -21,10 +21,16 @@ import org.apache.nifi.record.path.RecordPathEvaluationContext;
|
|||
import org.apache.nifi.record.path.StandardFieldValue;
|
||||
import org.apache.nifi.record.path.paths.RecordPathSegment;
|
||||
import org.apache.nifi.record.path.util.RecordPathUtils;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.temporal.ChronoField;
|
||||
import java.time.temporal.TemporalAccessor;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class ToDate extends RecordPathSegment {
|
||||
|
@ -53,20 +59,61 @@ public class ToDate extends RecordPathSegment {
|
|||
return fieldValues.filter(fv -> fv.getValue() != null)
|
||||
.map(fv -> {
|
||||
|
||||
if (!(fv.getValue() instanceof String)) {
|
||||
final Object fieldValue = fv.getValue();
|
||||
if (!(fieldValue instanceof String)) {
|
||||
return fv;
|
||||
}
|
||||
|
||||
final java.text.DateFormat dateFormat = getDateFormat(this.dateFormat, this.timeZoneID, context);
|
||||
final DateTimeFormatter dateTimeFormatter = getDateTimeFormatter(dateFormat, context);
|
||||
|
||||
final Date dateValue;
|
||||
try {
|
||||
dateValue = DataTypeUtils.toDate(fv.getValue(), () -> dateFormat, fv.getField().getFieldName());
|
||||
} catch (final Exception e) {
|
||||
return fv;
|
||||
}
|
||||
final TemporalAccessor parsed = dateTimeFormatter.parse(fieldValue.toString());
|
||||
|
||||
if (dateValue == null) {
|
||||
int year = 0;
|
||||
if (parsed.isSupported(ChronoField.YEAR_OF_ERA)) {
|
||||
year = parsed.get(ChronoField.YEAR_OF_ERA);
|
||||
}
|
||||
|
||||
int month = 0;
|
||||
if (parsed.isSupported(ChronoField.MONTH_OF_YEAR)) {
|
||||
month = parsed.get(ChronoField.MONTH_OF_YEAR);
|
||||
}
|
||||
|
||||
int day = 0;
|
||||
if (parsed.isSupported(ChronoField.DAY_OF_MONTH)) {
|
||||
day = parsed.get(ChronoField.DAY_OF_MONTH);
|
||||
}
|
||||
|
||||
int hour = 0;
|
||||
if (parsed.isSupported(ChronoField.HOUR_OF_DAY)) {
|
||||
hour = parsed.get(ChronoField.HOUR_OF_DAY);
|
||||
}
|
||||
|
||||
int minute = 0;
|
||||
if (parsed.isSupported(ChronoField.MINUTE_OF_HOUR)) {
|
||||
minute = parsed.get(ChronoField.MINUTE_OF_HOUR);
|
||||
}
|
||||
|
||||
int second = 0;
|
||||
if (parsed.isSupported(ChronoField.SECOND_OF_MINUTE)) {
|
||||
second = parsed.get(ChronoField.SECOND_OF_MINUTE);
|
||||
}
|
||||
|
||||
int nano = 0;
|
||||
if (parsed.isSupported(ChronoField.MILLI_OF_SECOND)) {
|
||||
nano = parsed.get(ChronoField.NANO_OF_SECOND);
|
||||
}
|
||||
|
||||
ZoneId zoneId = getZoneId(context);
|
||||
if (zoneId == null) {
|
||||
zoneId = ZoneId.systemDefault();
|
||||
}
|
||||
|
||||
final ZonedDateTime zonedDateTime = ZonedDateTime.of(year, month, day, hour, minute, second, nano, zoneId);
|
||||
final Instant instant = zonedDateTime.toInstant();
|
||||
dateValue = Date.from(instant);
|
||||
} catch (final Exception e) {
|
||||
return fv;
|
||||
}
|
||||
|
||||
|
@ -74,7 +121,7 @@ public class ToDate extends RecordPathSegment {
|
|||
});
|
||||
}
|
||||
|
||||
private java.text.DateFormat getDateFormat(final RecordPathSegment dateFormatSegment, final RecordPathSegment timeZoneID, final RecordPathEvaluationContext context) {
|
||||
private DateTimeFormatter getDateTimeFormatter(final RecordPathSegment dateFormatSegment, final RecordPathEvaluationContext context) {
|
||||
if (dateFormatSegment == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -85,18 +132,31 @@ public class ToDate extends RecordPathSegment {
|
|||
}
|
||||
|
||||
try {
|
||||
if (timeZoneID == null) {
|
||||
return DataTypeUtils.getDateFormat(dateFormatString);
|
||||
} else {
|
||||
final String timeZoneStr = RecordPathUtils.getFirstStringValue(timeZoneID, context);
|
||||
if (StringUtils.isEmpty(timeZoneStr)) {
|
||||
return null;
|
||||
}
|
||||
return DataTypeUtils.getDateFormat(dateFormatString, timeZoneStr);
|
||||
final ZoneId zoneId = getZoneId(context);
|
||||
final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(dateFormatString);
|
||||
if (zoneId != null) {
|
||||
formatter.withZone(zoneId);
|
||||
}
|
||||
return formatter;
|
||||
} catch (final Exception e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private ZoneId getZoneId(final RecordPathEvaluationContext context) {
|
||||
final ZoneId zoneId;
|
||||
|
||||
if (timeZoneID == null) {
|
||||
zoneId = null;
|
||||
} else {
|
||||
final String timeZoneStr = RecordPathUtils.getFirstStringValue(timeZoneID, context);
|
||||
if (StringUtils.isEmpty(timeZoneStr)) {
|
||||
zoneId = null;
|
||||
} else {
|
||||
zoneId = TimeZone.getTimeZone(timeZoneStr).toZoneId();
|
||||
}
|
||||
}
|
||||
|
||||
return zoneId;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.nifi.record.path;
|
|||
import java.nio.charset.IllegalCharsetNameException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.text.DateFormat;
|
||||
import java.text.ParseException;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
|
@ -55,6 +53,7 @@ import org.junit.jupiter.api.Test;
|
|||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
@ -86,10 +85,10 @@ public class TestRecordPath {
|
|||
|
||||
@Test
|
||||
public void testCompile() {
|
||||
System.out.println(RecordPath.compile("/person/name/last"));
|
||||
System.out.println(RecordPath.compile("/person[2]"));
|
||||
System.out.println(RecordPath.compile("//person[2]"));
|
||||
System.out.println(RecordPath.compile("/person/child[1]//sibling/name"));
|
||||
RecordPath.compile("/person/name/last");
|
||||
RecordPath.compile("/person[2]");
|
||||
RecordPath.compile("//person[2]");
|
||||
RecordPath.compile("/person/child[1]//sibling/name");
|
||||
|
||||
// contains is a 'filter function' so can be used as the predicate
|
||||
RecordPath.compile("/name[contains(., 'hello')]");
|
||||
|
@ -1461,20 +1460,22 @@ public class TestRecordPath {
|
|||
values.put("date", "2017-10-20T11:00:00Z");
|
||||
final Record record = new MapRecord(schema, values);
|
||||
|
||||
assertTrue(RecordPath.compile("toDate(/date, \"yyyy-MM-dd'T'HH:mm:ss'Z'\")").evaluate(record).getSelectedFields().findFirst().get().getValue() instanceof Date);
|
||||
assertTrue(RecordPath.compile("toDate(/date, \"yyyy-MM-dd'T'HH:mm:ss'Z'\", \"GMT+8:00\")").evaluate(record).getSelectedFields().findFirst().get().getValue() instanceof Date);
|
||||
final Object evaluated = RecordPath.compile("toDate(/date, \"yyyy-MM-dd'T'HH:mm:ss'Z'\")").evaluate(record).getSelectedFields().findFirst().get().getValue();
|
||||
assertInstanceOf(java.util.Date.class, evaluated);
|
||||
|
||||
final Object evaluatedTimeZone = RecordPath.compile("toDate(/date, \"yyyy-MM-dd'T'HH:mm:ss'Z'\", \"GMT+8:00\")").evaluate(record).getSelectedFields().findFirst().get().getValue();
|
||||
assertInstanceOf(java.util.Date.class, evaluatedTimeZone);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToDateFromLong() throws ParseException {
|
||||
public void testToDateFromLong() {
|
||||
final List<RecordField> fields = new ArrayList<>();
|
||||
fields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
|
||||
fields.add(new RecordField("date", RecordFieldType.LONG.getDataType()));
|
||||
|
||||
final RecordSchema schema = new SimpleRecordSchema(fields);
|
||||
|
||||
final DateFormat dateFormat = DataTypeUtils.getDateFormat("yyyy-MM-dd");
|
||||
final long dateValue = dateFormat.parse("2017-10-20T11:00:00Z").getTime();
|
||||
final long dateValue = 0L;
|
||||
|
||||
final Map<String, Object> values = new HashMap<>();
|
||||
values.put("id", 48);
|
||||
|
@ -1540,10 +1541,10 @@ public class TestRecordPath {
|
|||
assertEquals(adjustedDateTime, fieldValue3.getValue());
|
||||
|
||||
final FieldValue fieldValueUnchanged = RecordPath.compile("format( toDate(/date, \"yyyy-MM-dd'T'HH:mm:ss\"), 'INVALID' )").evaluate(record).getSelectedFields().findFirst().get();
|
||||
assertEquals(localDateFormatted, fieldValueUnchanged.getValue().toString());
|
||||
assertInstanceOf(java.util.Date.class, fieldValueUnchanged.getValue());
|
||||
final FieldValue fieldValueUnchanged2 = RecordPath.compile("format( toDate(/date, \"yyyy-MM-dd'T'HH:mm:ss\"), 'INVALID' , 'INVALID')")
|
||||
.evaluate(record).getSelectedFields().findFirst().get();
|
||||
assertEquals(localDateFormatted, fieldValueUnchanged2.getValue().toString());
|
||||
assertInstanceOf(java.util.Date.class, fieldValueUnchanged2.getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.nifi.serialization.record;
|
|||
|
||||
import org.apache.nifi.serialization.SchemaValidationException;
|
||||
import org.apache.nifi.serialization.SimpleRecordSchema;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.type.MapDataType;
|
||||
|
@ -29,7 +31,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.DateFormat;
|
||||
import java.time.LocalDate;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -42,7 +44,6 @@ import java.util.Map;
|
|||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class MapRecord implements Record {
|
||||
private static final Logger logger = LoggerFactory.getLogger(MapRecord.class);
|
||||
|
@ -225,7 +226,8 @@ public class MapRecord implements Record {
|
|||
return convertToString(getValue(fieldName), dataTypeOption.get().getFormat());
|
||||
}
|
||||
|
||||
return DataTypeUtils.toString(getValue(fieldName), (Supplier<DateFormat>) null);
|
||||
final FieldConverter<Object, String> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class);
|
||||
return converter.convertField(getValue(fieldName), Optional.empty(), fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -278,7 +280,9 @@ public class MapRecord implements Record {
|
|||
|
||||
@Override
|
||||
public Date getAsDate(final String fieldName, final String format) {
|
||||
return DataTypeUtils.toDate(getValue(fieldName), () -> DataTypeUtils.getDateFormat(format), fieldName);
|
||||
final FieldConverter<Object, LocalDate> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
final LocalDate localDate = converter.convertField(getValue(fieldName), Optional.ofNullable(format), fieldName);
|
||||
return localDate == null ? null : java.sql.Date.valueOf(localDate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Caching Registry containing DateTimeFormatter objects for internal use with Field Converters
|
||||
*/
|
||||
class DateTimeFormatterRegistry {
|
||||
private static final Map<String, DateTimeFormatter> FORMATTERS = new ConcurrentHashMap<>();
|
||||
|
||||
static DateTimeFormatter getDateTimeFormatter(final String pattern) {
|
||||
Objects.requireNonNull(pattern, "Pattern required");
|
||||
final DateTimeFormatter formatter = FORMATTERS.computeIfAbsent(pattern, DateTimeFormatter::ofPattern);
|
||||
FORMATTERS.putIfAbsent(pattern, formatter);
|
||||
return formatter;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
/**
|
||||
* Field Conversion Exception with standard properties
|
||||
*/
|
||||
public class FieldConversionException extends IllegalTypeConversionException {
|
||||
|
||||
protected FieldConversionException(final Class<?> outputFieldType, final Object field, final String fieldName) {
|
||||
super("Conversion not supported for [%s] named [%s] to [%s]".formatted(field, fieldName, outputFieldType.getName()));
|
||||
}
|
||||
|
||||
protected FieldConversionException(final Class<?> outputFieldType, final Object field, final String fieldName, final Throwable cause) {
|
||||
super("Conversion failed for [%s] named [%s] to [%s] [%s] %s".formatted(field, fieldName, outputFieldType.getName(), cause.getClass().getName(), cause.getMessage()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
/**
|
||||
* Registry for abstracting access to Field Converter implementations
|
||||
*/
|
||||
public interface FieldConverterRegistry {
|
||||
/**
|
||||
* Get registered Field Converter using specified Output Field Type Class
|
||||
*
|
||||
* @param outputFieldType Output Field Type Class
|
||||
* @return Field Converter from Object to Output Field Type
|
||||
* @param <T> Output Field Type
|
||||
*/
|
||||
<T> FieldConverter<Object, T> getFieldConverter(Class<T> outputFieldType);
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Convert Object to java.time.LocalDate using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
class ObjectLocalDateFieldConverter implements FieldConverter<Object, LocalDate> {
|
||||
/**
|
||||
* Convert Object field to java.sql.Timestamp using optional format supported in DateTimeFormatter
|
||||
*
|
||||
* @param field Field can be null or a supported input type
|
||||
* @param pattern Format pattern optional for parsing
|
||||
* @param name Field name for tracking
|
||||
* @return Timestamp or null when input field is null or empty string
|
||||
* @throws IllegalTypeConversionException Thrown on parsing failures or unsupported types of input fields
|
||||
*/
|
||||
@Override
|
||||
public LocalDate convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
if (field == null) {
|
||||
return null;
|
||||
}
|
||||
if (field instanceof LocalDate) {
|
||||
return (LocalDate) field;
|
||||
}
|
||||
if (field instanceof java.sql.Date date) {
|
||||
return date.toLocalDate();
|
||||
}
|
||||
if (field instanceof java.util.Date date) {
|
||||
final Instant instant = date.toInstant();
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof Number) {
|
||||
final Number number = (Number) field;
|
||||
final Instant instant = Instant.ofEpochMilli(number.longValue());
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof String) {
|
||||
final String string = field.toString().trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (pattern.isPresent()) {
|
||||
final DateTimeFormatter formatter = DateTimeFormatterRegistry.getDateTimeFormatter(pattern.get());
|
||||
try {
|
||||
return LocalDate.parse(string, formatter);
|
||||
} catch (final DateTimeParseException e) {
|
||||
throw new FieldConversionException(LocalDate.class, field, name, e);
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
final long number = Long.parseLong(string);
|
||||
final Instant instant = Instant.ofEpochMilli(number);
|
||||
return ofInstant(instant);
|
||||
} catch (final NumberFormatException e) {
|
||||
throw new FieldConversionException(LocalDate.class, field, name, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw new FieldConversionException(LocalDate.class, field, name);
|
||||
}
|
||||
|
||||
private LocalDate ofInstant(final Instant instant) {
|
||||
return LocalDate.ofInstant(instant, ZoneId.systemDefault());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Convert Object to java.time.LocalDateTime using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
class ObjectLocalDateTimeFieldConverter implements FieldConverter<Object, LocalDateTime> {
|
||||
/**
|
||||
* Convert Object field to java.sql.Timestamp using optional format supported in DateTimeFormatter
|
||||
*
|
||||
* @param field Field can be null or a supported input type
|
||||
* @param pattern Format pattern optional for parsing
|
||||
* @param name Field name for tracking
|
||||
* @return Timestamp or null when input field is null or empty string
|
||||
* @throws IllegalTypeConversionException Thrown on parsing failures or unsupported types of input fields
|
||||
*/
|
||||
@Override
|
||||
public LocalDateTime convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
if (field == null) {
|
||||
return null;
|
||||
}
|
||||
if (field instanceof LocalDateTime) {
|
||||
return (LocalDateTime) field;
|
||||
}
|
||||
if (field instanceof Date date) {
|
||||
final Instant instant = Instant.ofEpochMilli(date.getTime());
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof Number) {
|
||||
final Number number = (Number) field;
|
||||
final Instant instant = Instant.ofEpochMilli(number.longValue());
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof String) {
|
||||
final String string = field.toString().trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (pattern.isPresent()) {
|
||||
final DateTimeFormatter formatter = DateTimeFormatterRegistry.getDateTimeFormatter(pattern.get());
|
||||
try {
|
||||
return LocalDateTime.parse(string, formatter);
|
||||
} catch (final DateTimeParseException e) {
|
||||
throw new FieldConversionException(LocalDateTime.class, field, name, e);
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
final long number = Long.parseLong(string);
|
||||
final Instant instant = Instant.ofEpochMilli(number);
|
||||
return ofInstant(instant);
|
||||
} catch (final NumberFormatException e) {
|
||||
throw new FieldConversionException(LocalDateTime.class, field, name, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw new FieldConversionException(LocalDateTime.class, field, name);
|
||||
}
|
||||
|
||||
private LocalDateTime ofInstant(final Instant instant) {
|
||||
return LocalDateTime.ofInstant(instant, ZoneId.systemDefault());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
import java.sql.Time;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Convert Object to java.time.LocalTime using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
class ObjectLocalTimeFieldConverter implements FieldConverter<Object, LocalTime> {
|
||||
/**
|
||||
* Convert Object field to java.time.LocalTime using optional format supported in DateTimeFormatter
|
||||
*
|
||||
* @param field Field can be null or a supported input type
|
||||
* @param pattern Format pattern optional for parsing
|
||||
* @param name Field name for tracking
|
||||
* @return LocalTime or null when input field is null or empty string
|
||||
* @throws IllegalTypeConversionException Thrown on parsing failures or unsupported types of input fields
|
||||
*/
|
||||
@Override
|
||||
public LocalTime convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
if (field == null) {
|
||||
return null;
|
||||
}
|
||||
if (field instanceof LocalTime) {
|
||||
return (LocalTime) field;
|
||||
}
|
||||
if (field instanceof Time time) {
|
||||
// Convert to Instant preserving millisecond precision
|
||||
final long epochMilli = time.getTime();
|
||||
final Instant instant = Instant.ofEpochMilli(epochMilli);
|
||||
return LocalTime.ofInstant(instant, ZoneId.systemDefault());
|
||||
}
|
||||
if (field instanceof Date date) {
|
||||
return ofInstant(Instant.ofEpochMilli(date.getTime()));
|
||||
}
|
||||
if (field instanceof Number) {
|
||||
final Number number = (Number) field;
|
||||
final Instant instant = Instant.ofEpochMilli(number.longValue());
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof String) {
|
||||
final String string = field.toString().trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (pattern.isPresent()) {
|
||||
final DateTimeFormatter formatter = DateTimeFormatterRegistry.getDateTimeFormatter(pattern.get());
|
||||
try {
|
||||
return LocalTime.parse(string, formatter);
|
||||
} catch (final DateTimeParseException e) {
|
||||
throw new FieldConversionException(LocalTime.class, field, name, e);
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
final long number = Long.parseLong(string);
|
||||
final Instant instant = Instant.ofEpochMilli(number);
|
||||
return ofInstant(instant);
|
||||
} catch (final NumberFormatException e) {
|
||||
throw new FieldConversionException(LocalTime.class, field, name, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw new FieldConversionException(LocalTime.class, field, name);
|
||||
}
|
||||
|
||||
private LocalTime ofInstant(final Instant instant) {
|
||||
final ZonedDateTime zonedDateTime = instant.atZone(ZoneId.systemDefault());
|
||||
return zonedDateTime.toLocalTime();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
import java.time.Instant;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Convert Object to java.time.OffsetDateTime using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
class ObjectOffsetDateTimeFieldConverter implements FieldConverter<Object, OffsetDateTime> {
|
||||
/**
|
||||
* Convert Object field to java.time.OffsetDateTime using optional format supported in DateTimeFormatter
|
||||
*
|
||||
* @param field Field can be null or a supported input type
|
||||
* @param pattern Format pattern optional for parsing
|
||||
* @param name Field name for tracking
|
||||
* @return OffsetDateTime or null when input field is null or empty string
|
||||
* @throws IllegalTypeConversionException Thrown on parsing failures or unsupported types of input fields
|
||||
*/
|
||||
@Override
|
||||
public OffsetDateTime convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
if (field == null) {
|
||||
return null;
|
||||
}
|
||||
if (field instanceof OffsetDateTime) {
|
||||
return (OffsetDateTime) field;
|
||||
}
|
||||
if (field instanceof Date date) {
|
||||
final Instant instant = date.toInstant();
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof Number) {
|
||||
final Number number = (Number) field;
|
||||
final Instant instant = Instant.ofEpochMilli(number.longValue());
|
||||
return ofInstant(instant);
|
||||
}
|
||||
if (field instanceof String) {
|
||||
final String string = field.toString().trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (pattern.isPresent()) {
|
||||
final DateTimeFormatter formatter = DateTimeFormatterRegistry.getDateTimeFormatter(pattern.get());
|
||||
try {
|
||||
return OffsetDateTime.parse(string, formatter);
|
||||
} catch (final DateTimeParseException e) {
|
||||
throw new FieldConversionException(OffsetDateTime.class, field, name, e);
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
final long number = Long.parseLong(string);
|
||||
final Instant instant = Instant.ofEpochMilli(number);
|
||||
return ofInstant(instant);
|
||||
} catch (final NumberFormatException e) {
|
||||
throw new FieldConversionException(OffsetDateTime.class, field, name, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw new FieldConversionException(OffsetDateTime.class, field, name);
|
||||
}
|
||||
|
||||
private OffsetDateTime ofInstant(final Instant instant) {
|
||||
return OffsetDateTime.ofInstant(instant, ZoneId.systemDefault());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
import java.io.Reader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Clob;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Convert Object to String using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
class ObjectStringFieldConverter implements FieldConverter<Object, String> {
|
||||
private static final Map<String, DateTimeFormatter> FORMATTERS = new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Convert Object field to String using optional format supported in DateTimeFormatter
|
||||
*
|
||||
* @param field Field can be null or a supported input type
|
||||
* @param pattern Format pattern optional for parsing
|
||||
* @param name Field name for tracking
|
||||
* @return String or null when input field is null or empty string
|
||||
* @throws IllegalTypeConversionException Thrown on parsing failures or unsupported types of input fields
|
||||
*/
|
||||
@Override
|
||||
public String convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
if (field == null) {
|
||||
return null;
|
||||
}
|
||||
if (field instanceof String) {
|
||||
return field.toString();
|
||||
}
|
||||
if (field instanceof java.sql.Timestamp timestamp) {
|
||||
if (pattern.isEmpty()) {
|
||||
return Long.toString(timestamp.getTime());
|
||||
}
|
||||
final DateTimeFormatter formatter = DateTimeFormatterRegistry.getDateTimeFormatter(pattern.get());
|
||||
final LocalDateTime localDateTime = timestamp.toLocalDateTime();
|
||||
return formatter.format(localDateTime);
|
||||
}
|
||||
if (field instanceof java.util.Date date) {
|
||||
if (pattern.isEmpty()) {
|
||||
return Long.toString(date.getTime());
|
||||
}
|
||||
final DateTimeFormatter formatter = DateTimeFormatterRegistry.getDateTimeFormatter(pattern.get());
|
||||
// java.sql.Date and java.sql.Time do not support toInstant()
|
||||
final Instant instant = Instant.ofEpochMilli(date.getTime());
|
||||
final ZonedDateTime dateTime = instant.atZone(ZoneId.systemDefault());
|
||||
return formatter.format(dateTime);
|
||||
}
|
||||
if (field instanceof byte[] bytes) {
|
||||
return new String(bytes, StandardCharsets.UTF_16);
|
||||
}
|
||||
if (field instanceof Byte[] bytes) {
|
||||
final byte[] converted = new byte[bytes.length];
|
||||
for (int i = 0; i < bytes.length; i++) {
|
||||
converted[i] = bytes[i];
|
||||
}
|
||||
return new String(converted, StandardCharsets.UTF_16);
|
||||
}
|
||||
if (field instanceof Clob clob) {
|
||||
final StringBuilder builder = new StringBuilder();
|
||||
final char[] buffer = new char[32768];
|
||||
try (Reader reader = clob.getCharacterStream()) {
|
||||
int charsRead;
|
||||
while ((charsRead = reader.read(buffer)) != -1) {
|
||||
builder.append(buffer, 0, charsRead);
|
||||
}
|
||||
return builder.toString();
|
||||
} catch (final Exception e) {
|
||||
throw new FieldConversionException(String.class, field, name, e);
|
||||
}
|
||||
}
|
||||
return field.toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
|
||||
import java.sql.Time;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Convert Object to java.sql.Time using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
class ObjectTimeFieldConverter implements FieldConverter<Object, Time> {
|
||||
private static final ObjectLocalTimeFieldConverter CONVERTER = new ObjectLocalTimeFieldConverter();
|
||||
|
||||
/**
|
||||
* Convert Object field to java.sql.Time using optional format supported in DateTimeFormatter
|
||||
*
|
||||
* @param field Field can be null or a supported input type
|
||||
* @param pattern Format pattern optional for parsing
|
||||
* @param name Field name for tracking
|
||||
* @return Timestamp or null when input field is null or empty string
|
||||
* @throws IllegalTypeConversionException Thrown on parsing failures or unsupported types of input fields
|
||||
*/
|
||||
@Override
|
||||
public Time convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
final LocalTime localTime = CONVERTER.convertField(field, pattern, name);
|
||||
|
||||
final Time time;
|
||||
if (localTime == null) {
|
||||
time = null;
|
||||
} else {
|
||||
final ZonedDateTime zonedDateTime = localTime.atDate(LocalDate.ofEpochDay(0)).atZone(ZoneId.systemDefault());
|
||||
final long epochMilli = zonedDateTime.toInstant().toEpochMilli();
|
||||
time = new Time(epochMilli);
|
||||
}
|
||||
return time;
|
||||
}
|
||||
}
|
|
@ -20,15 +20,14 @@ import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
|||
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Convert Object to java.sql.Timestamp using instanceof evaluation and optional format pattern for DateTimeFormatter
|
||||
*/
|
||||
public class ObjectTimestampFieldConverter implements FieldConverter<Object, Timestamp> {
|
||||
class ObjectTimestampFieldConverter implements FieldConverter<Object, Timestamp> {
|
||||
private static final ObjectLocalDateTimeFieldConverter CONVERTER = new ObjectLocalDateTimeFieldConverter();
|
||||
|
||||
/**
|
||||
* Convert Object field to java.sql.Timestamp using optional format supported in DateTimeFormatter
|
||||
*
|
||||
|
@ -40,47 +39,7 @@ public class ObjectTimestampFieldConverter implements FieldConverter<Object, Tim
|
|||
*/
|
||||
@Override
|
||||
public Timestamp convertField(final Object field, final Optional<String> pattern, final String name) {
|
||||
if (field == null) {
|
||||
return null;
|
||||
}
|
||||
if (field instanceof Timestamp) {
|
||||
return (Timestamp) field;
|
||||
}
|
||||
if (field instanceof Date) {
|
||||
final Date date = (Date) field;
|
||||
return new Timestamp(date.getTime());
|
||||
}
|
||||
if (field instanceof Number) {
|
||||
final Number number = (Number) field;
|
||||
return new Timestamp(number.longValue());
|
||||
}
|
||||
if (field instanceof String) {
|
||||
final String string = field.toString().trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (pattern.isPresent()) {
|
||||
final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(pattern.get());
|
||||
try {
|
||||
final LocalDateTime localDateTime = LocalDateTime.parse(string, formatter);
|
||||
return Timestamp.valueOf(localDateTime);
|
||||
} catch (final DateTimeParseException e) {
|
||||
final String message = String.format("Convert Field Name [%s] Value [%s] to Timestamp LocalDateTime parsing failed: %s", name, field, e.getMessage());
|
||||
throw new IllegalTypeConversionException(message);
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
final long number = Long.parseLong(string);
|
||||
return new Timestamp(number);
|
||||
} catch (final NumberFormatException e) {
|
||||
final String message = String.format("Convert Field Name [%s] Value [%s] to Timestamp Long parsing failed: %s", name, field, e.getMessage());
|
||||
throw new IllegalTypeConversionException(message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final String message = String.format("Convert Field Name [%s] Value [%s] Class [%s] to Timestamp not supported", name, field, field.getClass());
|
||||
throw new IllegalTypeConversionException(message);
|
||||
final LocalDateTime localDateTime = CONVERTER.convertField(field, pattern, name);
|
||||
return localDateTime == null ? null : Timestamp.valueOf(localDateTime);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.LocalTime;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class StandardFieldConverterRegistry implements FieldConverterRegistry {
|
||||
private static final Map<Class<?>, FieldConverter<Object, ?>> FIELD_CONVERTERS = Map.of(
|
||||
Time.class, new ObjectTimeFieldConverter(),
|
||||
Timestamp.class, new ObjectTimestampFieldConverter(),
|
||||
LocalDate.class, new ObjectLocalDateFieldConverter(),
|
||||
LocalDateTime.class, new ObjectLocalDateTimeFieldConverter(),
|
||||
LocalTime.class, new ObjectLocalTimeFieldConverter(),
|
||||
OffsetDateTime.class, new ObjectOffsetDateTimeFieldConverter(),
|
||||
String.class, new ObjectStringFieldConverter()
|
||||
);
|
||||
|
||||
private static final StandardFieldConverterRegistry REGISTRY = new StandardFieldConverterRegistry();
|
||||
|
||||
private StandardFieldConverterRegistry() {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Field Converter Registry instance for access to configured Field Converters
|
||||
*
|
||||
* @return Field Converter Registry
|
||||
*/
|
||||
public static FieldConverterRegistry getRegistry() {
|
||||
return REGISTRY;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Field Converter for specified output field type class
|
||||
*
|
||||
* @param outputFieldType Output Field Type Class
|
||||
* @return Field Converter
|
||||
* @param <T> Output Field Type
|
||||
* @throws IllegalArgumentException Thrown when Field Converter not found for Output Field Type
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public <T> FieldConverter<Object, T> getFieldConverter(final Class<T> outputFieldType) {
|
||||
Objects.requireNonNull(outputFieldType, "Output Field Type required");
|
||||
|
||||
final FieldConverter<Object, ?> fieldConverter = FIELD_CONVERTERS.get(outputFieldType);
|
||||
if (fieldConverter == null) {
|
||||
throw new IllegalArgumentException("Field Converter not found for Output Field Type [%s]".formatted(outputFieldType));
|
||||
}
|
||||
|
||||
return (FieldConverter<Object, T>) fieldConverter;
|
||||
}
|
||||
}
|
|
@ -24,6 +24,8 @@ import org.apache.nifi.serialization.record.Record;
|
|||
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.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.type.DecimalDataType;
|
||||
|
@ -48,14 +50,13 @@ import java.sql.Time;
|
|||
import java.sql.Timestamp;
|
||||
import java.sql.Types;
|
||||
import java.text.DateFormat;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -71,14 +72,12 @@ import java.util.Objects;
|
|||
import java.util.Optional;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
||||
public class DataTypeUtils {
|
||||
private static final Logger logger = LoggerFactory.getLogger(DataTypeUtils.class);
|
||||
|
||||
|
@ -113,9 +112,9 @@ public class DataTypeUtils {
|
|||
private static final Pattern FLOATING_POINT_PATTERN = Pattern.compile(doubleRegex);
|
||||
private static final Pattern DECIMAL_PATTERN = Pattern.compile(decimalRegex);
|
||||
|
||||
private static final Supplier<DateFormat> DEFAULT_DATE_FORMAT = () -> getDateFormat(RecordFieldType.DATE.getDefaultFormat());
|
||||
private static final Supplier<DateFormat> DEFAULT_TIME_FORMAT = () -> getDateFormat(RecordFieldType.TIME.getDefaultFormat());
|
||||
private static final Supplier<DateFormat> DEFAULT_TIMESTAMP_FORMAT = () -> getDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat());
|
||||
private static final Optional<String> DEFAULT_DATE_FORMAT = Optional.of(RecordFieldType.DATE.getDefaultFormat());
|
||||
private static final Optional<String> DEFAULT_TIME_FORMAT = Optional.of(RecordFieldType.TIME.getDefaultFormat());
|
||||
private static final Optional<String> DEFAULT_TIMESTAMP_FORMAT = Optional.of(RecordFieldType.TIMESTAMP.getDefaultFormat());
|
||||
|
||||
private static final int FLOAT_SIGNIFICAND_PRECISION = 24; // As specified in IEEE 754 binary32
|
||||
private static final int DOUBLE_SIGNIFICAND_PRECISION = 53; // As specified in IEEE 754 binary64
|
||||
|
@ -154,27 +153,40 @@ public class DataTypeUtils {
|
|||
return convertType(value, dataType, DEFAULT_DATE_FORMAT, DEFAULT_TIME_FORMAT, DEFAULT_TIMESTAMP_FORMAT, fieldName, charset);
|
||||
}
|
||||
|
||||
public static DateFormat getDateFormat(final RecordFieldType fieldType, final Supplier<DateFormat> dateFormat,
|
||||
final Supplier<DateFormat> timeFormat, final Supplier<DateFormat> timestampFormat) {
|
||||
private static String getDateFormat(final RecordFieldType fieldType, final Optional<String> dateFormat,
|
||||
final Optional<String> timeFormat, final Optional<String> timestampFormat) {
|
||||
switch (fieldType) {
|
||||
case DATE:
|
||||
return dateFormat.get();
|
||||
return dateFormat.orElse(null);
|
||||
case TIME:
|
||||
return timeFormat.get();
|
||||
return timeFormat.orElse(null);
|
||||
case TIMESTAMP:
|
||||
return timestampFormat.get();
|
||||
return timestampFormat.orElse(null);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
public static Object convertType(final Object value, final DataType dataType, final Supplier<DateFormat> dateFormat, final Supplier<DateFormat> timeFormat,
|
||||
final Supplier<DateFormat> timestampFormat, final String fieldName) {
|
||||
public static Object convertType(
|
||||
final Object value,
|
||||
final DataType dataType,
|
||||
final Optional<String> dateFormat,
|
||||
final Optional<String> timeFormat,
|
||||
final Optional<String> timestampFormat,
|
||||
final String fieldName
|
||||
) {
|
||||
return convertType(value, dataType, dateFormat, timeFormat, timestampFormat, fieldName, StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
public static Object convertType(final Object value, final DataType dataType, final Supplier<DateFormat> dateFormat, final Supplier<DateFormat> timeFormat,
|
||||
final Supplier<DateFormat> timestampFormat, final String fieldName, final Charset charset) {
|
||||
public static Object convertType(
|
||||
final Object value,
|
||||
final DataType dataType,
|
||||
final Optional<String> dateFormat,
|
||||
final Optional<String> timeFormat,
|
||||
final Optional<String> timestampFormat,
|
||||
final String fieldName,
|
||||
final Charset charset
|
||||
) {
|
||||
|
||||
if (value == null) {
|
||||
return null;
|
||||
|
@ -190,7 +202,9 @@ public class DataTypeUtils {
|
|||
case CHAR:
|
||||
return toCharacter(value, fieldName);
|
||||
case DATE:
|
||||
return convertTypeToDate(value, dateFormat, fieldName);
|
||||
final FieldConverter<Object, LocalDate> localDateConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
final LocalDate localDate = localDateConverter.convertField(value, dateFormat, fieldName);
|
||||
return localDate == null ? null : Date.valueOf(localDate);
|
||||
case DECIMAL:
|
||||
return toBigDecimal(value, fieldName);
|
||||
case DOUBLE:
|
||||
|
@ -206,11 +220,15 @@ public class DataTypeUtils {
|
|||
case ENUM:
|
||||
return toEnum(value, (EnumDataType) dataType, fieldName);
|
||||
case STRING:
|
||||
return toString(value, () -> getDateFormat(dataType.getFieldType(), dateFormat, timeFormat, timestampFormat), charset);
|
||||
final FieldConverter<Object, String> stringConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class);
|
||||
final String pattern = getDateFormat(dataType.getFieldType(), dateFormat, timeFormat, timestampFormat);
|
||||
return stringConverter.convertField(value, Optional.ofNullable(pattern), fieldName);
|
||||
case TIME:
|
||||
return toTime(value, timeFormat, fieldName);
|
||||
final FieldConverter<Object, Time> timeConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Time.class);
|
||||
return timeConverter.convertField(value, timeFormat, fieldName);
|
||||
case TIMESTAMP:
|
||||
return toTimestamp(value, timestampFormat, fieldName);
|
||||
final FieldConverter<Object, Timestamp> timestampConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class);
|
||||
return timestampConverter.convertField(value, timestampFormat, fieldName);
|
||||
case UUID:
|
||||
return toUUID(value);
|
||||
case ARRAY:
|
||||
|
@ -226,7 +244,7 @@ public class DataTypeUtils {
|
|||
final DataType chosenDataType = chooseDataType(value, choiceDataType);
|
||||
if (chosenDataType == null) {
|
||||
throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass()
|
||||
+ " for field " + fieldName + " to any of the following available Sub-Types for a Choice: " + choiceDataType.getPossibleSubTypes());
|
||||
+ " for field " + fieldName + " to any of the following available Sub-Types for a Choice: " + choiceDataType.getPossibleSubTypes());
|
||||
}
|
||||
|
||||
return convertType(value, chosenDataType, fieldName, charset);
|
||||
|
@ -938,12 +956,7 @@ public class DataTypeUtils {
|
|||
return value != null && (value instanceof Map || value instanceof MapRecord);
|
||||
}
|
||||
|
||||
|
||||
public static String toString(final Object value, final Supplier<DateFormat> format) {
|
||||
return toString(value, format, StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
public static String toString(final Object value, final Supplier<DateFormat> format, final Charset charset) {
|
||||
private static String toString(final Object value, final Supplier<DateFormat> format, final Charset charset) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -1029,17 +1042,18 @@ public class DataTypeUtils {
|
|||
return String.valueOf(((java.util.Date) value).getTime());
|
||||
}
|
||||
|
||||
final FieldConverter<Object, String> fieldConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class);
|
||||
if (value instanceof java.sql.Date) {
|
||||
return getDateFormat(format).format((java.util.Date) value);
|
||||
return fieldConverter.convertField(value, Optional.of(format), null);
|
||||
}
|
||||
if (value instanceof java.sql.Time) {
|
||||
return getDateFormat(format).format((java.util.Date) value);
|
||||
return fieldConverter.convertField(value, Optional.of(format), null);
|
||||
}
|
||||
if (value instanceof java.sql.Timestamp) {
|
||||
return getDateFormat(format).format((java.util.Date) value);
|
||||
return fieldConverter.convertField(value, Optional.of(format), null);
|
||||
}
|
||||
if (value instanceof java.util.Date) {
|
||||
return getDateFormat(format).format((java.util.Date) value);
|
||||
return fieldConverter.convertField(value, Optional.of(format), null);
|
||||
}
|
||||
if (value instanceof Blob) {
|
||||
Blob blob = (Blob) value;
|
||||
|
@ -1096,178 +1110,6 @@ public class DataTypeUtils {
|
|||
throw new IllegalTypeConversionException("Cannot convert value " + value + " of type " + dataType + " for field " + fieldName);
|
||||
}
|
||||
|
||||
public static java.sql.Date toDate(final Object value, final Supplier<DateFormat> format, final String fieldName) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (value instanceof Date) {
|
||||
return (Date) value;
|
||||
}
|
||||
|
||||
if (value instanceof java.util.Date) {
|
||||
java.util.Date _temp = (java.util.Date)value;
|
||||
return new Date(_temp.getTime());
|
||||
}
|
||||
|
||||
if (value instanceof Number) {
|
||||
final long longValue = ((Number) value).longValue();
|
||||
return new Date(longValue);
|
||||
}
|
||||
|
||||
if (value instanceof String) {
|
||||
try {
|
||||
final String string = ((String) value).trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (format == null) {
|
||||
return new Date(Long.parseLong(string));
|
||||
}
|
||||
|
||||
final DateFormat dateFormat = format.get();
|
||||
if (dateFormat == null) {
|
||||
return new Date(Long.parseLong(string));
|
||||
}
|
||||
final java.util.Date utilDate = dateFormat.parse(string);
|
||||
return new Date(utilDate.getTime());
|
||||
} catch (final ParseException | NumberFormatException e) {
|
||||
throw new IllegalTypeConversionException("Could not convert value [" + value
|
||||
+ "] of type java.lang.String to Date because the value is not in the expected date format: " + format + " for field " + fieldName);
|
||||
}
|
||||
}
|
||||
|
||||
throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Date for field " + fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Date Time Formatter using Zone Identifier
|
||||
*
|
||||
* @param pattern Date Format Pattern
|
||||
* @param zoneId Time Zone Identifier
|
||||
* @return Date Time Formatter or null when provided pattern is null
|
||||
*/
|
||||
public static DateTimeFormatter getDateTimeFormatter(final String pattern, final ZoneId zoneId) {
|
||||
if (pattern == null || zoneId == null) {
|
||||
return null;
|
||||
}
|
||||
return DateTimeFormatter.ofPattern(pattern).withZone(zoneId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert value to Local Date with support for conversion from numbers or formatted strings
|
||||
*
|
||||
* @param value Value to be converted
|
||||
* @param formatter Supplier for Date Time Formatter can be null when string parsing is not necessary
|
||||
* @param fieldName Field Name for value to be converted
|
||||
* @return Local Date or null when value to be converted is null
|
||||
* @throws IllegalTypeConversionException Thrown when conversion from string fails or unsupported value provided
|
||||
*/
|
||||
public static LocalDate toLocalDate(final Object value, final Supplier<DateTimeFormatter> formatter, final String fieldName) {
|
||||
LocalDate localDate;
|
||||
|
||||
if (value == null) {
|
||||
return null;
|
||||
} else if (value instanceof LocalDate) {
|
||||
localDate = (LocalDate) value;
|
||||
} else if (value instanceof java.sql.Date) {
|
||||
final java.sql.Date date = (java.sql.Date) value;
|
||||
localDate = date.toLocalDate();
|
||||
} else if (value instanceof java.util.Date) {
|
||||
final java.util.Date date = (java.util.Date) value;
|
||||
localDate = parseLocalDateEpochMillis(date.getTime());
|
||||
} else if (value instanceof Number) {
|
||||
final long epochMillis = ((Number) value).longValue();
|
||||
localDate = parseLocalDateEpochMillis(epochMillis);
|
||||
} else if (value instanceof String) {
|
||||
try {
|
||||
localDate = parseLocalDate((String) value, formatter);
|
||||
} catch (final RuntimeException e) {
|
||||
final String message = String.format("Failed Conversion of Field [%s] from String [%s] to LocalDate", fieldName, value);
|
||||
throw new IllegalTypeConversionException(message, e);
|
||||
}
|
||||
} else {
|
||||
final String message = String.format("Failed Conversion of Field [%s] from Value [%s] Type [%s] to LocalDate", fieldName, value, value.getClass());
|
||||
throw new IllegalTypeConversionException(message);
|
||||
}
|
||||
|
||||
return localDate;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert value to java.sql.Date using java.time.LocalDate parsing and conversion from DateFormat to DateTimeFormatter
|
||||
*
|
||||
* Transitional method supporting conversion from legacy java.text.DateFormat to java.time.DateTimeFormatter
|
||||
*
|
||||
* @param value Value object to be converted
|
||||
* @param format Supplier function for java.text.DateFormat when necessary for parsing
|
||||
* @param fieldName Field name being parsed
|
||||
* @return java.sql.Date or null when value is null
|
||||
*/
|
||||
private static Date convertTypeToDate(final Object value, final Supplier<DateFormat> format, final String fieldName) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
} else {
|
||||
final LocalDate localDate = toLocalDate(value, () -> {
|
||||
final SimpleDateFormat dateFormat = (SimpleDateFormat) format.get();
|
||||
return dateFormat == null ? null : DateTimeFormatter.ofPattern(dateFormat.toPattern());
|
||||
}, fieldName);
|
||||
return Date.valueOf(localDate);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse Local Date from String using Date Time Formatter when supplied
|
||||
*
|
||||
* @param value String not null containing either formatted string or number of epoch milliseconds
|
||||
* @param formatter Supplier for Date Time Formatter
|
||||
* @return Local Date or null when provided value is empty
|
||||
*/
|
||||
private static LocalDate parseLocalDate(final String value, final Supplier<DateTimeFormatter> formatter) {
|
||||
LocalDate localDate = null;
|
||||
|
||||
final String normalized = value.trim();
|
||||
if (!normalized.isEmpty()) {
|
||||
if (formatter == null) {
|
||||
localDate = parseLocalDateEpochMillis(normalized);
|
||||
} else {
|
||||
final DateTimeFormatter dateTimeFormatter = formatter.get();
|
||||
if (dateTimeFormatter == null) {
|
||||
localDate = parseLocalDateEpochMillis(normalized);
|
||||
} else {
|
||||
localDate = LocalDate.parse(normalized, dateTimeFormatter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return localDate;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Parse Local Date from string expected to contain number of epoch milliseconds
|
||||
*
|
||||
* @param number Number string expected to contain epoch milliseconds
|
||||
* @return Local Date converted from epoch milliseconds
|
||||
*/
|
||||
private static LocalDate parseLocalDateEpochMillis(final String number) {
|
||||
final long epochMillis = Long.parseLong(number);
|
||||
return parseLocalDateEpochMillis(epochMillis);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse Local Date from epoch milliseconds using System Default Zone Offset
|
||||
*
|
||||
* @param epochMillis Epoch milliseconds
|
||||
* @return Local Date converted from epoch milliseconds
|
||||
*/
|
||||
private static LocalDate parseLocalDateEpochMillis(final long epochMillis) {
|
||||
final Instant instant = Instant.ofEpochMilli(epochMillis);
|
||||
final ZonedDateTime zonedDateTime = instant.atZone(ZoneOffset.systemDefault());
|
||||
return zonedDateTime.toLocalDate();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a java.sql.Date object in local time zone (typically coming from a java.sql.ResultSet and having 00:00:00 time part)
|
||||
* to UTC normalized form (storing the epoch corresponding to the UTC time with the same date/time as the input).
|
||||
|
@ -1296,9 +1138,9 @@ public class DataTypeUtils {
|
|||
}
|
||||
|
||||
try {
|
||||
getDateFormat(format).parse((String) value);
|
||||
DateTimeFormatter.ofPattern(format).parse(value.toString());
|
||||
return true;
|
||||
} catch (final ParseException e) {
|
||||
} catch (final DateTimeParseException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -1320,147 +1162,14 @@ public class DataTypeUtils {
|
|||
return true;
|
||||
}
|
||||
|
||||
public static Time toTime(final Object value, final Supplier<DateFormat> format, final String fieldName) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (value instanceof Time) {
|
||||
return (Time) value;
|
||||
}
|
||||
|
||||
if (value instanceof Number) {
|
||||
final long longValue = ((Number) value).longValue();
|
||||
return new Time(longValue);
|
||||
}
|
||||
|
||||
if (value instanceof String) {
|
||||
try {
|
||||
final String string = ((String) value).trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (format == null) {
|
||||
return new Time(Long.parseLong(string));
|
||||
}
|
||||
|
||||
final DateFormat dateFormat = format.get();
|
||||
if (dateFormat == null) {
|
||||
return new Time(Long.parseLong(string));
|
||||
}
|
||||
final java.util.Date utilDate = dateFormat.parse(string);
|
||||
return new Time(utilDate.getTime());
|
||||
} catch (final ParseException e) {
|
||||
throw new IllegalTypeConversionException("Could not convert value [" + value
|
||||
+ "] of type java.lang.String to Time for field " + fieldName + " because the value is not in the expected date format: " + format);
|
||||
}
|
||||
}
|
||||
|
||||
throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Time for field " + fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Date Format using default Local Time Zone
|
||||
*
|
||||
* @param pattern Date Format Pattern used for new SimpleDateFormat()
|
||||
* @return Date Format or null when pattern not provided
|
||||
*/
|
||||
public static DateFormat getDateFormat(final String pattern) {
|
||||
if (pattern == null) {
|
||||
return null;
|
||||
}
|
||||
return getDateFormat(pattern, TimeZone.getDefault());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Date Format using specified Time Zone to adjust Date during processing
|
||||
*
|
||||
* @param pattern Date Format Pattern used for new SimpleDateFormat()
|
||||
* @param timeZoneId Time Zone Identifier used for TimeZone.getTimeZone()
|
||||
* @return Date Format or null when input parameters not provided
|
||||
*/
|
||||
public static DateFormat getDateFormat(final String pattern, final String timeZoneId) {
|
||||
if (pattern == null || timeZoneId == null) {
|
||||
return null;
|
||||
}
|
||||
return getDateFormat(pattern, TimeZone.getTimeZone(timeZoneId));
|
||||
}
|
||||
|
||||
private static DateFormat getDateFormat(final String pattern, final TimeZone timeZone) {
|
||||
if (pattern == null) {
|
||||
return null;
|
||||
}
|
||||
final DateFormat dateFormat = new SimpleDateFormat(pattern);
|
||||
dateFormat.setTimeZone(timeZone);
|
||||
return dateFormat;
|
||||
}
|
||||
|
||||
public static boolean isTimeTypeCompatible(final Object value, final String format) {
|
||||
return isDateTypeCompatible(value, format);
|
||||
}
|
||||
|
||||
public static Timestamp toTimestamp(final Object value, final Supplier<DateFormat> format, final String fieldName) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (value instanceof Timestamp) {
|
||||
return (Timestamp) value;
|
||||
}
|
||||
|
||||
if (value instanceof java.util.Date) {
|
||||
return new Timestamp(((java.util.Date)value).getTime());
|
||||
}
|
||||
|
||||
if (value instanceof Number) {
|
||||
final long longValue = ((Number) value).longValue();
|
||||
return new Timestamp(longValue);
|
||||
}
|
||||
|
||||
if (value instanceof String) {
|
||||
final String string = ((String) value).trim();
|
||||
if (string.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
try {
|
||||
if (format == null) {
|
||||
return new Timestamp(Long.parseLong(string));
|
||||
}
|
||||
|
||||
final DateFormat dateFormat = format.get();
|
||||
if (dateFormat == null) {
|
||||
return new Timestamp(Long.parseLong(string));
|
||||
}
|
||||
|
||||
final java.util.Date utilDate = dateFormat.parse(string);
|
||||
return new Timestamp(utilDate.getTime());
|
||||
} catch (final ParseException e) {
|
||||
final DateFormat dateFormat = format.get();
|
||||
final String formatDescription;
|
||||
if (dateFormat == null) {
|
||||
formatDescription = "Numeric";
|
||||
} else if (dateFormat instanceof SimpleDateFormat) {
|
||||
formatDescription = ((SimpleDateFormat) dateFormat).toPattern();
|
||||
} else {
|
||||
formatDescription = dateFormat.toString();
|
||||
}
|
||||
|
||||
throw new IllegalTypeConversionException("Could not convert value [" + value
|
||||
+ "] of type java.lang.String to Timestamp for field " + fieldName + " because the value is not in the expected date format: "
|
||||
+ formatDescription);
|
||||
}
|
||||
}
|
||||
|
||||
throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Timestamp for field " + fieldName);
|
||||
}
|
||||
|
||||
public static boolean isTimestampTypeCompatible(final Object value, final String format) {
|
||||
return isDateTypeCompatible(value, format);
|
||||
}
|
||||
|
||||
|
||||
public static BigInteger toBigInt(final Object value, final String fieldName) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.nifi.serialization.record;
|
|||
import org.apache.nifi.serialization.SimpleRecordSchema;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.DecimalDataType;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
|
@ -265,7 +264,6 @@ public class ResultSetRecordSetTest {
|
|||
|
||||
@Test
|
||||
public void testCreateRecord() throws SQLException {
|
||||
// given
|
||||
final RecordSchema recordSchema = givenRecordSchema(COLUMNS);
|
||||
|
||||
LocalDate testDate = LocalDate.of(2021, 1, 26);
|
||||
|
@ -311,11 +309,9 @@ public class ResultSetRecordSetTest {
|
|||
when(resultSet.getObject(COLUMN_NAME_BIG_DECIMAL_4)).thenReturn(bigDecimal4Value);
|
||||
when(resultSet.getObject(COLUMN_NAME_BIG_DECIMAL_5)).thenReturn(bigDecimal5Value);
|
||||
|
||||
// when
|
||||
ResultSetRecordSet testSubject = new ResultSetRecordSet(resultSet, recordSchema);
|
||||
Record record = testSubject.createRecord(resultSet);
|
||||
|
||||
// then
|
||||
assertEquals(varcharValue, record.getAsString(COLUMN_NAME_VARCHAR));
|
||||
assertEquals(bigintValue, record.getAsLong(COLUMN_NAME_BIGINT));
|
||||
assertEquals(rowidValue, record.getAsLong(COLUMN_NAME_ROWID));
|
||||
|
@ -324,7 +320,8 @@ public class ResultSetRecordSetTest {
|
|||
assertEquals(charValue, record.getValue(COLUMN_NAME_CHAR));
|
||||
|
||||
assertEquals(dateValue, record.getAsDate(COLUMN_NAME_DATE, null));
|
||||
assertEquals(timestampValue, DataTypeUtils.toTimestamp(record.getValue(COLUMN_NAME_TIMESTAMP), null, COLUMN_NAME_TIMESTAMP));
|
||||
final Object timestampObject = record.getValue(COLUMN_NAME_TIMESTAMP);
|
||||
assertEquals(timestampValue, timestampObject);
|
||||
|
||||
assertEquals(integerValue, record.getAsInt(COLUMN_NAME_INTEGER));
|
||||
assertEquals(doubleValue, record.getAsDouble(COLUMN_NAME_DOUBLE));
|
||||
|
|
|
@ -29,17 +29,11 @@ import java.math.BigInteger;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.sql.Timestamp;
|
||||
import java.sql.Types;
|
||||
import java.text.DateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.LocalTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -49,7 +43,6 @@ import java.util.LinkedList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.DoubleAdder;
|
||||
|
@ -66,8 +59,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
|||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestDataTypeUtils {
|
||||
private static final ZoneId SYSTEM_DEFAULT_ZONE_ID = ZoneOffset.systemDefault();
|
||||
|
||||
private static final String ISO_8601_YEAR_MONTH_DAY = "2000-01-01";
|
||||
|
||||
private static final String CUSTOM_MONTH_DAY_YEAR = "01-01-2000";
|
||||
|
@ -76,45 +67,12 @@ public class TestDataTypeUtils {
|
|||
|
||||
private static final String DATE_FIELD = "date";
|
||||
|
||||
/**
|
||||
* This is a unit test to verify conversion java Date objects to Timestamps. Support for this was
|
||||
* required in order to help the MongoDB packages handle date/time logical types in the Record API.
|
||||
*/
|
||||
@Test
|
||||
public void testDateToTimestamp() {
|
||||
java.util.Date date = new java.util.Date();
|
||||
Timestamp ts = DataTypeUtils.toTimestamp(date, null, null);
|
||||
|
||||
assertNotNull(ts);
|
||||
assertEquals(ts.getTime(), date.getTime(), "Times didn't match");
|
||||
|
||||
java.sql.Date sDate = new java.sql.Date(date.getTime());
|
||||
ts = DataTypeUtils.toTimestamp(date, null, null);
|
||||
assertNotNull(ts);
|
||||
assertEquals(ts.getTime(), sDate.getTime(), "Times didn't match");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntDoubleWiderType() {
|
||||
assertEquals(Optional.of(RecordFieldType.DOUBLE.getDataType()), DataTypeUtils.getWiderType(RecordFieldType.INT.getDataType(), RecordFieldType.DOUBLE.getDataType()));
|
||||
assertEquals(Optional.of(RecordFieldType.DOUBLE.getDataType()), DataTypeUtils.getWiderType(RecordFieldType.DOUBLE.getDataType(), RecordFieldType.INT.getDataType()));
|
||||
}
|
||||
|
||||
/*
|
||||
* This was a bug in NiFi 1.8 where converting from a Timestamp to a Date with the record path API
|
||||
* would throw an exception.
|
||||
*/
|
||||
@Test
|
||||
public void testTimestampToDate() {
|
||||
java.util.Date date = new java.util.Date();
|
||||
Timestamp ts = DataTypeUtils.toTimestamp(date, null, null);
|
||||
assertNotNull(ts);
|
||||
|
||||
java.sql.Date output = DataTypeUtils.toDate(ts, null, null);
|
||||
assertNotNull(output);
|
||||
assertEquals(output.getTime(), ts.getTime(), "Timestamps didn't match");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertRecordMapToJavaMap() {
|
||||
assertNull(DataTypeUtils.convertRecordMapToJavaMap(null, null));
|
||||
|
@ -1063,61 +1021,15 @@ public class TestDataTypeUtils {
|
|||
assertEquals(ISO_8601_YEAR_MONTH_DAY, converted.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert String to java.sql.Date using custom pattern DateFormat with configured GMT Time Zone
|
||||
*/
|
||||
@Test
|
||||
public void testConvertTypeStringToDateConfiguredTimeZoneFormat() {
|
||||
final DateFormat dateFormat = DataTypeUtils.getDateFormat(CUSTOM_MONTH_DAY_YEAR_PATTERN, "GMT");
|
||||
final Object converted = DataTypeUtils.convertType(CUSTOM_MONTH_DAY_YEAR, RecordFieldType.DATE.getDataType(), () -> dateFormat, null, null,"date");
|
||||
assertTrue(converted instanceof java.sql.Date, "Converted value is not java.sql.Date");
|
||||
assertEquals(ISO_8601_YEAR_MONTH_DAY, converted.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert String to java.sql.Date using custom pattern DateFormat with system default Time Zone
|
||||
*/
|
||||
@Test
|
||||
public void testConvertTypeStringToDateConfiguredSystemDefaultTimeZoneFormat() {
|
||||
final DateFormat dateFormat = DataTypeUtils.getDateFormat(CUSTOM_MONTH_DAY_YEAR_PATTERN, TimeZone.getDefault().getID());
|
||||
final Object converted = DataTypeUtils.convertType(CUSTOM_MONTH_DAY_YEAR, RecordFieldType.DATE.getDataType(), () -> dateFormat, null, null,"date");
|
||||
final Object converted = DataTypeUtils.convertType(
|
||||
CUSTOM_MONTH_DAY_YEAR, RecordFieldType.DATE.getDataType(), Optional.of(CUSTOM_MONTH_DAY_YEAR_PATTERN), Optional.empty(), Optional.empty(),"date"
|
||||
);
|
||||
assertTrue(converted instanceof java.sql.Date, "Converted value is not java.sql.Date");
|
||||
assertEquals(ISO_8601_YEAR_MONTH_DAY, converted.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToLocalDateFromString() {
|
||||
assertToLocalDateEquals(ISO_8601_YEAR_MONTH_DAY, ISO_8601_YEAR_MONTH_DAY);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToLocalDateFromSqlDate() {
|
||||
assertToLocalDateEquals(ISO_8601_YEAR_MONTH_DAY, java.sql.Date.valueOf(ISO_8601_YEAR_MONTH_DAY));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToLocalDateFromUtilDate() {
|
||||
final LocalDate localDate = LocalDate.parse(ISO_8601_YEAR_MONTH_DAY);
|
||||
final long epochMillis = toEpochMilliSystemDefaultZone(localDate);
|
||||
assertToLocalDateEquals(ISO_8601_YEAR_MONTH_DAY, new java.util.Date(epochMillis));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToLocalDateFromNumberEpochMillis() {
|
||||
final LocalDate localDate = LocalDate.parse(ISO_8601_YEAR_MONTH_DAY);
|
||||
final long epochMillis = toEpochMilliSystemDefaultZone(localDate);
|
||||
assertToLocalDateEquals(ISO_8601_YEAR_MONTH_DAY, epochMillis);
|
||||
}
|
||||
|
||||
private long toEpochMilliSystemDefaultZone(final LocalDate localDate) {
|
||||
final LocalTime localTime = LocalTime.of(0, 0);
|
||||
final Instant instantSystemDefaultZone = ZonedDateTime.of(localDate, localTime, SYSTEM_DEFAULT_ZONE_ID).toInstant();
|
||||
return instantSystemDefaultZone.toEpochMilli();
|
||||
}
|
||||
|
||||
private void assertToLocalDateEquals(final String expected, final Object value) {
|
||||
final DateTimeFormatter systemDefaultZoneFormatter = DataTypeUtils.getDateTimeFormatter(RecordFieldType.DATE.getDefaultFormat(), SYSTEM_DEFAULT_ZONE_ID);
|
||||
final LocalDate localDate = DataTypeUtils.toLocalDate(value, () -> systemDefaultZoneFormatter, DATE_FIELD);
|
||||
assertEquals(expected, localDate.toString(), String.format("Value Class [%s] to LocalDate not matched", value.getClass()));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.RecordFieldType;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.sql.Timestamp;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class ObjectOffsetDateTimeFieldConverterTest {
|
||||
private static final ObjectOffsetDateTimeFieldConverter CONVERTER = new ObjectOffsetDateTimeFieldConverter();
|
||||
|
||||
private static final String DEFAULT_PATTERN = RecordFieldType.TIMESTAMP.getDefaultFormat();
|
||||
|
||||
private static final String FIELD_NAME = OffsetDateTime.class.getSimpleName();
|
||||
|
||||
private static final String EMPTY = "";
|
||||
|
||||
private static final String DATE_TIME_DEFAULT = "2000-01-01 12:00:00";
|
||||
|
||||
private static final String OFFSET_DATE_TIME_DEFAULT = "2000-01-01T12:30:45Z";
|
||||
|
||||
private static final String OFFSET_DATE_TIME_PATTERN = "yyyy-MM-dd'T'HH:mm:ssX";
|
||||
|
||||
private static final String NANOSECONDS_PATTERN = "yyyy-MM-dd'T'HH:mm:ss.SSSSSSSSSX";
|
||||
|
||||
private static final String DATE_TIME_NANOSECONDS = "2000-01-01T12:00:00.123456789Z";
|
||||
|
||||
@Test
|
||||
public void testConvertFieldNull() {
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(null, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertNull(offsetDateTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldTimestamp() {
|
||||
final Timestamp field = new Timestamp(System.currentTimeMillis());
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(field, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertEquals(field.getTime(), offsetDateTime.toInstant().toEpochMilli());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldDate() {
|
||||
final Date field = new Date();
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(field, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertEquals(field.getTime(), offsetDateTime.toInstant().toEpochMilli());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldLong() {
|
||||
final long field = System.currentTimeMillis();
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(field, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertEquals(field, offsetDateTime.toInstant().toEpochMilli());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringEmpty() {
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(EMPTY, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertNull(offsetDateTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringFormatNull() {
|
||||
final long currentTime = System.currentTimeMillis();
|
||||
final String field = Long.toString(currentTime);
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(field, Optional.empty(), FIELD_NAME);
|
||||
assertEquals(currentTime, offsetDateTime.toInstant().toEpochMilli());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringFormatNullNumberFormatException() {
|
||||
final String field = String.class.getSimpleName();
|
||||
final FieldConversionException exception = assertThrows(FieldConversionException.class, () -> CONVERTER.convertField(field, Optional.empty(), FIELD_NAME));
|
||||
assertTrue(exception.getMessage().contains(field));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringFormatDefault() {
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(OFFSET_DATE_TIME_DEFAULT, Optional.of(OFFSET_DATE_TIME_PATTERN), FIELD_NAME);
|
||||
assertEquals(OFFSET_DATE_TIME_DEFAULT, offsetDateTime.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringFormatCustomNanoseconds() {
|
||||
final OffsetDateTime offsetDateTime = CONVERTER.convertField(DATE_TIME_NANOSECONDS, Optional.of(NANOSECONDS_PATTERN), FIELD_NAME);
|
||||
final OffsetDateTime expected = OffsetDateTime.parse(DATE_TIME_NANOSECONDS);
|
||||
assertEquals(expected, offsetDateTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringFormatCustomFormatterException() {
|
||||
final FieldConversionException exception = assertThrows(FieldConversionException.class, () -> CONVERTER.convertField(DATE_TIME_DEFAULT, Optional.of(NANOSECONDS_PATTERN), FIELD_NAME));
|
||||
assertTrue(exception.getMessage().contains(DATE_TIME_DEFAULT));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.RecordFieldType;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.sql.Time;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
||||
public class ObjectTimeFieldConverterTest {
|
||||
private static final ObjectTimeFieldConverter CONVERTER = new ObjectTimeFieldConverter();
|
||||
|
||||
private static final String DEFAULT_PATTERN = RecordFieldType.TIME.getDefaultFormat();
|
||||
|
||||
private static final String FIELD_NAME = Time.class.getSimpleName();
|
||||
|
||||
private static final String EMPTY = "";
|
||||
|
||||
private static final String TIME_DEFAULT = "12:30:45";
|
||||
|
||||
private static final String TIME_NANOSECONDS_PATTERN = "HH:mm:ss.SSSSSSSSS";
|
||||
|
||||
private static final String TIME_NANOSECONDS = "12:30:45.123456789";
|
||||
|
||||
@Test
|
||||
public void testConvertFieldNull() {
|
||||
final Time time = CONVERTER.convertField(null, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertNull(time);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldStringEmpty() {
|
||||
final Time time = CONVERTER.convertField(EMPTY, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertNull(time);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldTime() {
|
||||
final Time field = Time.valueOf(TIME_DEFAULT);
|
||||
final Time time = CONVERTER.convertField(field, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertEquals(field.getTime(), time.getTime());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldTimeNanoseconds() {
|
||||
final Time time = CONVERTER.convertField(TIME_NANOSECONDS, Optional.of(TIME_NANOSECONDS_PATTERN), FIELD_NAME);
|
||||
assertEquals(TIME_DEFAULT, time.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldDate() {
|
||||
final Date field = new Date();
|
||||
final Time time = CONVERTER.convertField(field, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertNotNull(time);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertFieldLong() {
|
||||
final long field = System.currentTimeMillis();
|
||||
final Time time = CONVERTER.convertField(field, Optional.of(DEFAULT_PATTERN), FIELD_NAME);
|
||||
assertNotNull(time);
|
||||
}
|
||||
}
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.nifi.serialization.record.field;
|
||||
|
||||
import org.apache.nifi.serialization.record.RecordFieldType;
|
||||
import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.sql.Timestamp;
|
||||
|
@ -88,7 +87,7 @@ public class ObjectTimestampFieldConverterTest {
|
|||
@Test
|
||||
public void testConvertFieldStringFormatNullNumberFormatException() {
|
||||
final String field = String.class.getSimpleName();
|
||||
final IllegalTypeConversionException exception = assertThrows(IllegalTypeConversionException.class, () -> CONVERTER.convertField(field, Optional.empty(), FIELD_NAME));
|
||||
final FieldConversionException exception = assertThrows(FieldConversionException.class, () -> CONVERTER.convertField(field, Optional.empty(), FIELD_NAME));
|
||||
assertTrue(exception.getMessage().contains(field));
|
||||
}
|
||||
|
||||
|
@ -108,7 +107,7 @@ public class ObjectTimestampFieldConverterTest {
|
|||
|
||||
@Test
|
||||
public void testConvertFieldStringFormatCustomFormatterException() {
|
||||
final IllegalTypeConversionException exception = assertThrows(IllegalTypeConversionException.class, () -> CONVERTER.convertField(DATE_TIME_DEFAULT, DATE_TIME_NANOSECONDS_PATTERN, FIELD_NAME));
|
||||
final FieldConversionException exception = assertThrows(FieldConversionException.class, () -> CONVERTER.convertField(DATE_TIME_DEFAULT, DATE_TIME_NANOSECONDS_PATTERN, FIELD_NAME));
|
||||
assertTrue(exception.getMessage().contains(DATE_TIME_DEFAULT));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,17 +32,15 @@ import javax.security.auth.login.LoginException;
|
|||
import java.security.PrivilegedAction;
|
||||
import java.security.PrivilegedActionException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.time.Instant;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* Base class for implementations of KerberosUser.
|
||||
*
|
||||
* Generally implementations must provide the specific Configuration instance for performing the login,
|
||||
* along with an optional CallbackHandler.
|
||||
*
|
||||
* Some functionality in this class is adapted from Hadoop's UserGroupInformation.
|
||||
*/
|
||||
public abstract class AbstractKerberosUser implements KerberosUser {
|
||||
|
@ -50,6 +48,7 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractKerberosUser.class);
|
||||
|
||||
static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss'Z'";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(DATE_FORMAT);
|
||||
|
||||
/**
|
||||
* Percentage of the ticket window to use before we renew the TGT.
|
||||
|
@ -99,7 +98,7 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
|
||||
loginContext.login();
|
||||
loggedIn.set(true);
|
||||
LOGGER.debug("Successful login for {}", new Object[]{principal});
|
||||
LOGGER.debug("Successful login for {}", principal);
|
||||
} catch (final LoginException le) {
|
||||
throw new KerberosLoginException("Unable to login with " + principal + " due to: " + le.getMessage(), le);
|
||||
}
|
||||
|
@ -143,7 +142,7 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
try {
|
||||
loginContext.logout();
|
||||
loggedIn.set(false);
|
||||
LOGGER.debug("Successful logout for {}", new Object[]{principal});
|
||||
LOGGER.debug("Successful logout for {}", principal);
|
||||
|
||||
loginContext = null;
|
||||
} catch (final LoginException e) {
|
||||
|
@ -190,7 +189,6 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
/**
|
||||
* Re-login a user from keytab if TGT is expired or is close to expiry.
|
||||
*
|
||||
* @throws LoginException if an error happens performing the re-login
|
||||
*/
|
||||
@Override
|
||||
public synchronized boolean checkTGTAndRelogin() {
|
||||
|
@ -200,7 +198,7 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
return logoutAndLogin();
|
||||
}
|
||||
|
||||
if (tgt != null && System.currentTimeMillis() < getRefreshTime(tgt)) {
|
||||
if (System.currentTimeMillis() < getRefreshTime(tgt)) {
|
||||
LOGGER.debug("TGT for {} was found, but has not reached expiration window", principal);
|
||||
return false;
|
||||
}
|
||||
|
@ -222,7 +220,7 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
}
|
||||
|
||||
private boolean logoutAndLogin() {
|
||||
LOGGER.debug("Performing logout/login", principal);
|
||||
LOGGER.debug("Performing logout/login {}", principal);
|
||||
logout();
|
||||
login();
|
||||
return true;
|
||||
|
@ -271,15 +269,14 @@ public abstract class AbstractKerberosUser implements KerberosUser {
|
|||
final long end = tgt.getEndTime().getTime();
|
||||
|
||||
if (LOGGER.isTraceEnabled()) {
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_FORMAT);
|
||||
final String startDate = dateFormat.format(new Date(start));
|
||||
final String endDate = dateFormat.format(new Date(end));
|
||||
final String startDate = DATE_TIME_FORMATTER.format(Instant.ofEpochMilli(start));
|
||||
final String endDate = DATE_TIME_FORMATTER.format(Instant.ofEpochMilli(end));
|
||||
LOGGER.trace("TGT for {} is valid starting at [{}]", principal, startDate);
|
||||
LOGGER.trace("TGT for {} expires at [{}]", principal, endDate);
|
||||
if (tgt.getRenewTill() == null) {
|
||||
LOGGER.trace("TGT for {} is non-renewable", principal);
|
||||
} else {
|
||||
LOGGER.trace("TGT for {} renews until [{}]", principal, dateFormat.format(tgt.getRenewTill()));
|
||||
LOGGER.trace("TGT for {} renews until [{}]", principal, DATE_TIME_FORMATTER.format(tgt.getRenewTill().toInstant()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -447,7 +447,7 @@ public class FormatUtils {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse text to Instant - support different formats like: zoned date time, date time, date, time (similar to those supported in SimpleDateFormat)
|
||||
* Parse text to Instant - support different formats like: zoned date time, date time, date, time
|
||||
* @param formatter configured formatter
|
||||
* @param text text which will be parsed
|
||||
* @return parsed Instant
|
||||
|
|
|
@ -16,20 +16,19 @@
|
|||
*/
|
||||
package org.apache.nifi.util.text;
|
||||
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
|
||||
class SimpleDateFormatMatcher implements DateTimeMatcher {
|
||||
private final DateFormat dateFormat;
|
||||
class DateTimeFormatterMatcher implements DateTimeMatcher {
|
||||
private final DateTimeFormatter dateTimeFormatter;
|
||||
|
||||
public SimpleDateFormatMatcher(final String format) {
|
||||
this.dateFormat = new SimpleDateFormat(format);
|
||||
public DateTimeFormatterMatcher(final String format) {
|
||||
this.dateTimeFormatter = DateTimeFormatter.ofPattern(format);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(final String text) {
|
||||
try {
|
||||
dateFormat.parse(text);
|
||||
dateTimeFormatter.parse(text);
|
||||
return true;
|
||||
} catch (final Exception e) {
|
||||
return false;
|
|
@ -19,13 +19,13 @@ package org.apache.nifi.util.text;
|
|||
/**
|
||||
* <p>
|
||||
* A utility class that can be used to determine whether or not a String matches a given date/time format, as specified
|
||||
* by the Time Format used in {@link java.text.SimpleDateFormat}. It is not uncommon to see code written along the lines of:
|
||||
* by the Time Format used in {@link java.time.format.DateTimeFormatter}. It is not uncommon to see code written along the lines of:
|
||||
* </p>
|
||||
*
|
||||
* <code><pre>
|
||||
* final String format = "yyyy/MM/dd HH:mm:ss.SSS";
|
||||
* try {
|
||||
* new SimpleDateFormat(format).parse(text);
|
||||
* DateTimeFormatter.ofPattern(format).parse(text);
|
||||
* return true;
|
||||
* } catch (Exception e) {
|
||||
* return false;
|
||||
|
@ -34,7 +34,7 @@ package org.apache.nifi.util.text;
|
|||
*
|
||||
* <p>
|
||||
* This approach, however, is frowned upon for two important reasons. Firstly, the performance is poor. A micro-benchmark that involves executing
|
||||
* the above code (even reusing the SimpleDateFormat object) to evaluate whether or not <code>text</code> is a timestamp took approximately 125-130 seconds
|
||||
* the above code o evaluate whether or not <code>text</code> is a timestamp took approximately 125-130 seconds
|
||||
* to iterate 1,000,000 times (after discarding the first 1,000,000 iterations as a 'warmup'). As a comparison, this utility takes about 8-11 seconds against
|
||||
* the same data and on the same machine.
|
||||
* </p>
|
||||
|
@ -46,8 +46,8 @@ package org.apache.nifi.util.text;
|
|||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Note, however, that this class is not intended to replace SimpleDateFormat, as it does not perform the actual parsing but instead only determines whether or not
|
||||
* a given input text matches the pattern, so that if it does, a SimpleDateFormat can be used parse the input.
|
||||
* Note, however, that this class is not intended to replace DateTimeFormatter, as it does not perform the actual parsing but instead only determines whether or not
|
||||
* a given input text matches the pattern, so that if it does, a DateTimeFormatter can be used parse the input.
|
||||
* </p>
|
||||
*/
|
||||
public interface DateTimeMatcher {
|
||||
|
|
|
@ -49,8 +49,8 @@ class DateTimeMatcherCompiler {
|
|||
|
||||
matchers.add(regexMatcher);
|
||||
|
||||
// Use the SimpleDateFormatMatcher only if our regex matches. This allows us to parse the date only to guarantee that we are correct if we say that the input text matches.
|
||||
matchers.add(new SimpleDateFormatMatcher(format));
|
||||
// Use the DateTimeFormatterMatcher only if our regex matches. This allows us to parse the date only to guarantee that we are correct if we say that the input text matches.
|
||||
matchers.add(new DateTimeFormatterMatcher(format));
|
||||
return new ListDateTimeMatcher(matchers);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -487,7 +487,7 @@ public class RegexDateTimeMatcher implements DateTimeMatcher {
|
|||
}
|
||||
|
||||
private String getGMTOffsetTimeZone() {
|
||||
// From SimpleDateFormat JavaDocs, GMTOffsetTimeZone defined as: GMT Sign Hours : Minutes
|
||||
// From Date Format JavaDocs, GMTOffsetTimeZone defined as: GMT Sign Hours : Minutes
|
||||
// Sign defined as '-' or '+'
|
||||
// Hours defined as 1 or 2 digits, Minutes defined as 1 or 2 digits
|
||||
// Digit defined as number between 0-9
|
||||
|
|
|
@ -25,14 +25,12 @@ import org.junit.jupiter.params.provider.NullSource;
|
|||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.text.DecimalFormatSymbols;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Arrays;
|
||||
import java.util.Locale;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Stream;
|
||||
|
@ -404,8 +402,8 @@ public class TestFormatUtils {
|
|||
|
||||
@ParameterizedTest
|
||||
@MethodSource("getParseToInstantUsingFormatterWithoutZones")
|
||||
public void testParseToInstantUsingFormatterWithoutZones(String pattern, String parsedDateTime, String systemDefaultZoneId, String expectedUtcDateTime) throws Exception {
|
||||
checkSameResultsWithSimpleDateFormat(pattern, parsedDateTime, systemDefaultZoneId, null, expectedUtcDateTime);
|
||||
public void testParseToInstantUsingFormatterWithoutZones(String pattern, String parsedDateTime, String systemDefaultZoneId, String expectedUtcDateTime) {
|
||||
checkSameResultsWithFormatter(pattern, parsedDateTime, systemDefaultZoneId, null, expectedUtcDateTime);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> getParseToInstantUsingFormatterWithoutZones() {
|
||||
|
@ -428,8 +426,8 @@ public class TestFormatUtils {
|
|||
|
||||
@ParameterizedTest
|
||||
@MethodSource("getParseToInstantUsingFormatterWithZone")
|
||||
public void testParseToInstantUsingFormatterWithZone(String pattern, String parsedDateTime, String systemDefaultZoneId, String formatZoneId, String expectedUtcDateTime) throws Exception {
|
||||
checkSameResultsWithSimpleDateFormat(pattern, parsedDateTime, systemDefaultZoneId, formatZoneId, expectedUtcDateTime);
|
||||
public void testParseToInstantUsingFormatterWithZone(String pattern, String parsedDateTime, String systemDefaultZoneId, String formatZoneId, String expectedUtcDateTime) {
|
||||
checkSameResultsWithFormatter(pattern, parsedDateTime, systemDefaultZoneId, formatZoneId, expectedUtcDateTime);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> getParseToInstantUsingFormatterWithZone() {
|
||||
|
@ -448,8 +446,8 @@ public class TestFormatUtils {
|
|||
|
||||
@ParameterizedTest
|
||||
@MethodSource("getParseToInstantWithZonePassedInText")
|
||||
public void testParseToInstantWithZonePassedInText(String pattern, String parsedDateTime, String systemDefaultZoneId, String expectedUtcDateTime) throws Exception {
|
||||
checkSameResultsWithSimpleDateFormat(pattern, parsedDateTime, systemDefaultZoneId, null, expectedUtcDateTime);
|
||||
public void testParseToInstantWithZonePassedInText(String pattern, String parsedDateTime, String systemDefaultZoneId, String expectedUtcDateTime) {
|
||||
checkSameResultsWithFormatter(pattern, parsedDateTime, systemDefaultZoneId, null, expectedUtcDateTime);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> getParseToInstantWithZonePassedInText() {
|
||||
|
@ -465,27 +463,19 @@ public class TestFormatUtils {
|
|||
Arguments.of(pattern, "2020-01-01 02:00:00 +0000", UTC_TIME_ZONE_ID, "2020-01-01T02:00:00"));
|
||||
}
|
||||
|
||||
private void checkSameResultsWithSimpleDateFormat(String pattern, String parsedDateTime, String systemDefaultZoneId, String formatZoneId, String expectedUtcDateTime) throws Exception {
|
||||
private void checkSameResultsWithFormatter(String pattern, String parsedDateTime, String systemDefaultZoneId, String formatZoneId, String expectedUtcDateTime) {
|
||||
TimeZone current = TimeZone.getDefault();
|
||||
TimeZone.setDefault(TimeZone.getTimeZone(systemDefaultZoneId));
|
||||
try {
|
||||
checkSameResultsWithSimpleDateFormat(pattern, parsedDateTime, formatZoneId, expectedUtcDateTime);
|
||||
checkSameResultsWithFormatter(pattern, parsedDateTime, formatZoneId, expectedUtcDateTime);
|
||||
} finally {
|
||||
TimeZone.setDefault(current);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkSameResultsWithSimpleDateFormat(String pattern, String parsedDateTime, String formatterZoneId, String expectedUtcDateTime) throws Exception {
|
||||
private void checkSameResultsWithFormatter(String pattern, String parsedDateTime, String formatterZoneId, String expectedUtcDateTime) {
|
||||
Instant expectedInstant = LocalDateTime.parse(expectedUtcDateTime).atZone(ZoneOffset.UTC).toInstant();
|
||||
|
||||
// reference implementation
|
||||
SimpleDateFormat sdf = new SimpleDateFormat(pattern, Locale.US);
|
||||
if (formatterZoneId != null) {
|
||||
sdf.setTimeZone(TimeZone.getTimeZone(formatterZoneId));
|
||||
}
|
||||
Instant simpleDateFormatResult = sdf.parse(parsedDateTime).toInstant();
|
||||
assertEquals(expectedInstant, simpleDateFormatResult);
|
||||
|
||||
// current implementation
|
||||
DateTimeFormatter dtf = FormatUtils.prepareLenientCaseInsensitiveDateTimeFormatter(pattern);
|
||||
if (formatterZoneId != null) {
|
||||
|
|
|
@ -45,8 +45,6 @@ public class TestRegexDateTimeMatcher {
|
|||
exampleToPattern.put("12 Dec 2018", "dd MMM yyyy");
|
||||
exampleToPattern.put("12 December 2018", "dd MMM yyyy");
|
||||
|
||||
// TODO: The following examples are taken from the SimpleDateFormat's JavaDoc. Ensure that this is not a licensing concern,
|
||||
// since it is not being distributed.
|
||||
exampleToPattern.put("2001.07.04 AD at 12:08:56 PDT", "yyyy.MM.dd G 'at' HH:mm:ss z");
|
||||
exampleToPattern.put("Wed, Jul 4, '01", "EEE, MMM d, ''yy");
|
||||
exampleToPattern.put("12:08 PM", "h:mm a");
|
||||
|
|
|
@ -640,7 +640,7 @@ Converts a Date to a String in the given format with an optional time zone. The
|
|||
time zone when the second argument is not provided.
|
||||
|
||||
The first argument to this function must be a Date or a Number, and the second argument must be a format String that
|
||||
follows the Java SimpleDateFormat, and the third argument, optional, must be a format String that
|
||||
follows the Java DateTimeFormatter, and the third argument, optional, must be a format String that
|
||||
either an abbreviation such as "PST", a full name such as "America/Los_Angeles", or a custom ID such as "GMT-8:00"
|
||||
|
||||
For example, given a schema such as:
|
||||
|
|
|
@ -17,8 +17,9 @@
|
|||
package org.apache.nifi.bundle;
|
||||
|
||||
import java.io.File;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Date;
|
||||
|
||||
/**
|
||||
|
@ -107,10 +108,9 @@ public class BundleDetails {
|
|||
public Date getBuildTimestampDate() {
|
||||
if (buildTimestamp != null && !buildTimestamp.isEmpty()) {
|
||||
try {
|
||||
SimpleDateFormat buildTimestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'");
|
||||
Date buildTimestampDate = buildTimestampFormat.parse(buildTimestamp);
|
||||
return buildTimestampDate;
|
||||
} catch (ParseException parseEx) {
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");
|
||||
return Date.from(OffsetDateTime.parse(buildTimestamp, dateTimeFormatter).toInstant());
|
||||
} catch (DateTimeParseException e) {
|
||||
return null;
|
||||
}
|
||||
} else {
|
||||
|
|
|
@ -39,8 +39,8 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
@ -59,6 +59,7 @@ public class RuntimeManifestGenerator {
|
|||
private static final String BUILD_TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss'Z'";
|
||||
private static final String BUILD_JDK = "Build-Jdk";
|
||||
private static final String BUILD_JDK_VENDOR = "Build-Jdk-Vendor";
|
||||
private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern(BUILD_TIMESTAMP_FORMAT);
|
||||
|
||||
private final File extensionManifestBaseDir;
|
||||
private final File buildPropertiesFile;
|
||||
|
@ -87,9 +88,7 @@ public class RuntimeManifestGenerator {
|
|||
|
||||
long buildTimestampMillis;
|
||||
try {
|
||||
final SimpleDateFormat buildTimestampFormat = new SimpleDateFormat(BUILD_TIMESTAMP_FORMAT);
|
||||
final Date buildTimestampDate = buildTimestampFormat.parse(buildTimestamp);
|
||||
buildTimestampMillis = buildTimestampDate.getTime();
|
||||
buildTimestampMillis = OffsetDateTime.parse(buildTimestamp, TIMESTAMP_FORMATTER).toInstant().toEpochMilli();
|
||||
} catch (Exception e) {
|
||||
buildTimestampMillis = System.currentTimeMillis();
|
||||
}
|
||||
|
|
|
@ -71,8 +71,6 @@ import java.io.Serializable;
|
|||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -877,7 +875,6 @@ public class PutS3Object extends AbstractS3Processor {
|
|||
|
||||
private final Lock s3BucketLock = new ReentrantLock();
|
||||
private final AtomicLong lastS3AgeOff = new AtomicLong(0L);
|
||||
private final DateFormat logFormat = new SimpleDateFormat();
|
||||
|
||||
protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3 s3, final long now, String bucket) {
|
||||
MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now, bucket);
|
||||
|
@ -940,10 +937,10 @@ public class PutS3Object extends AbstractS3Processor {
|
|||
try {
|
||||
s3.abortMultipartUpload(abortRequest);
|
||||
getLogger().info("Aborting out of date multipart upload, bucket {} key {} ID {}, initiated {}",
|
||||
new Object[]{bucket, uploadKey, uploadId, logFormat.format(upload.getInitiated())});
|
||||
bucket, uploadKey, uploadId, upload.getInitiated());
|
||||
} catch (AmazonClientException ace) {
|
||||
getLogger().info("Error trying to abort multipart upload from bucket {} with key {} and ID {}: {}",
|
||||
new Object[]{bucket, uploadKey, uploadId, ace.getMessage()});
|
||||
bucket, uploadKey, uploadId, ace.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -49,8 +49,8 @@ import org.mockito.ArgumentCaptor;
|
|||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
|
@ -175,8 +175,8 @@ public class TestListS3 {
|
|||
|
||||
runner.assertAllFlowFilesTransferred(ListS3.REL_SUCCESS, 1);
|
||||
|
||||
final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
|
||||
final String lastModifiedString = dateFormat.format(lastModified);
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");
|
||||
final String lastModifiedString = dateTimeFormatter.format(lastModified.toInstant().atZone(ZoneOffset.systemDefault()));
|
||||
|
||||
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(ListS3.REL_SUCCESS).get(0);
|
||||
flowFile.assertAttributeEquals("record.count", "3");
|
||||
|
|
|
@ -37,19 +37,18 @@ import javax.json.JsonBuilderFactory;
|
|||
import javax.json.JsonObject;
|
||||
import javax.json.JsonObjectBuilder;
|
||||
import javax.json.JsonValue;
|
||||
import java.io.IOException;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.text.DateFormat;
|
||||
import java.text.MessageFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
|
||||
@Tags({ "azure", "provenace", "reporting", "log analytics" })
|
||||
|
@ -59,6 +58,7 @@ public class AzureLogAnalyticsProvenanceReportingTask extends AbstractAzureLogAn
|
|||
protected static final String LAST_EVENT_ID_KEY = "last_event_id";
|
||||
protected static final String DESTINATION_URL_PATH = "/nifi";
|
||||
protected static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(TIMESTAMP_FORMAT);
|
||||
|
||||
static final PropertyDescriptor LOG_ANALYTICS_CUSTOM_LOG_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Log Analytics Custom Log Name").description("Log Analytics Custom Log Name").required(false)
|
||||
|
@ -278,7 +278,7 @@ public class AzureLogAnalyticsProvenanceReportingTask extends AbstractAzureLogAn
|
|||
}
|
||||
}
|
||||
|
||||
public void processProvenanceData(final ReportingContext context) throws IOException {
|
||||
public void processProvenanceData(final ReportingContext context) {
|
||||
getLogger().debug("Starting to process provenance data");
|
||||
final String workspaceId = context.getProperty(LOG_ANALYTICS_WORKSPACE_ID)
|
||||
.evaluateAttributeExpressions().getValue();
|
||||
|
@ -307,8 +307,6 @@ public class AzureLogAnalyticsProvenanceReportingTask extends AbstractAzureLogAn
|
|||
final Map<String, Object> config = Collections.emptyMap();
|
||||
final JsonBuilderFactory factory = Json.createBuilderFactory(config);
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
final DateFormat df = new SimpleDateFormat(TIMESTAMP_FORMAT);
|
||||
df.setTimeZone(TimeZone.getTimeZone("Z"));
|
||||
CreateConsumer(context);
|
||||
consumer.consumeEvents(context, (mapHolder, events) -> {
|
||||
StringBuilder stringBuilder = new StringBuilder();
|
||||
|
@ -318,7 +316,7 @@ public class AzureLogAnalyticsProvenanceReportingTask extends AbstractAzureLogAn
|
|||
final String processGroupId = mapHolder.getProcessGroupId(event.getComponentId(),
|
||||
event.getComponentType());
|
||||
final String processGroupName = mapHolder.getComponentName(processGroupId);
|
||||
final JsonObject jo = serialize(factory, builder, event, df, componentName,
|
||||
final JsonObject jo = serialize(factory, builder, event, componentName,
|
||||
processGroupId, processGroupName, hostname, url, rootGroupName,
|
||||
platform, nodeId, allowNullValues);
|
||||
stringBuilder.append(jo.toString());
|
||||
|
@ -345,7 +343,7 @@ public class AzureLogAnalyticsProvenanceReportingTask extends AbstractAzureLogAn
|
|||
}
|
||||
|
||||
private JsonObject serialize(final JsonBuilderFactory factory, final JsonObjectBuilder builder,
|
||||
final ProvenanceEventRecord event, final DateFormat df, final String componentName,
|
||||
final ProvenanceEventRecord event, final String componentName,
|
||||
final String processGroupId, final String processGroupName, final String hostname,
|
||||
final URL nifiUrl, final String applicationName, final String platform,
|
||||
final String nodeIdentifier, Boolean allowNullValues) {
|
||||
|
@ -353,7 +351,7 @@ public class AzureLogAnalyticsProvenanceReportingTask extends AbstractAzureLogAn
|
|||
addField(builder, "eventOrdinal", event.getEventId(), allowNullValues);
|
||||
addField(builder, "eventType", event.getEventType().name(), allowNullValues);
|
||||
addField(builder, "timestampMillis", event.getEventTime(), allowNullValues);
|
||||
addField(builder, "timestamp", df.format(event.getEventTime()), allowNullValues);
|
||||
addField(builder, "timestamp", DATE_TIME_FORMATTER.format(Instant.ofEpochMilli(event.getEventTime()).atOffset(ZoneOffset.UTC)), allowNullValues);
|
||||
addField(builder, "durationMillis", event.getEventDuration(), allowNullValues);
|
||||
addField(builder, "lineageStart", event.getLineageStartDate(), allowNullValues);
|
||||
addField(builder, "details", event.getDetails(), allowNullValues);
|
||||
|
|
|
@ -59,7 +59,9 @@ import org.apache.nifi.util.StopWatch;
|
|||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -71,7 +73,6 @@ import java.util.Map;
|
|||
import java.util.NoSuchElementException;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -181,7 +182,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
|
|||
.addValidator((subject, input, context) -> {
|
||||
final ValidationResult.Builder vrb = new ValidationResult.Builder().subject(subject).input(input);
|
||||
try {
|
||||
new SimpleDateFormat(input).format(new Date());
|
||||
DateTimeFormatter.ofPattern(input);
|
||||
vrb.valid(true).explanation("Valid date format pattern");
|
||||
} catch (Exception ex) {
|
||||
vrb.valid(false).explanation("the pattern is invalid: " + ex.getMessage());
|
||||
|
@ -526,9 +527,9 @@ public class QueryCassandra extends AbstractCassandraProcessor {
|
|||
final String dateFormatPattern = context
|
||||
.map(_context -> _context.getProperty(TIMESTAMP_FORMAT_PATTERN).getValue())
|
||||
.orElse(TIMESTAMP_FORMAT_PATTERN.getDefaultValue());
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat(dateFormatPattern);
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
return dateFormat.format(value);
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(dateFormatPattern);
|
||||
final OffsetDateTime offsetDateTime = value.toInstant().atOffset(ZoneOffset.UTC);
|
||||
return dateTimeFormatter.format(offsetDateTime);
|
||||
}
|
||||
|
||||
public static long convertToJsonStream(final ResultSet rs, long maxRowsPerFlowFile,
|
||||
|
@ -682,9 +683,8 @@ public class QueryCassandra extends AbstractCassandraProcessor {
|
|||
*
|
||||
* @param rs The result set from which an Avro schema will be created
|
||||
* @return An Avro schema corresponding to the given result set's metadata
|
||||
* @throws IOException If an error occurs during schema discovery/building
|
||||
*/
|
||||
public static Schema createSchema(final ResultSet rs) throws IOException {
|
||||
public static Schema createSchema(final ResultSet rs) {
|
||||
final ColumnDefinitions columnDefinitions = rs.getColumnDefinitions();
|
||||
final int nrOfColumns = (columnDefinitions == null ? 0 : columnDefinitions.size());
|
||||
String tableName = "NiFi_Cassandra_Query_Record";
|
||||
|
|
|
@ -26,7 +26,8 @@ import com.google.common.util.concurrent.ListenableFuture;
|
|||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
import java.util.List;
|
||||
|
@ -67,7 +68,7 @@ public class CassandraQueryTestUtil {
|
|||
"user_id", "first_name", "last_name", "emails", "top_places", "todo", "registered", "scale", "metric");
|
||||
|
||||
@Override
|
||||
public String answer(InvocationOnMock invocationOnMock) throws Throwable {
|
||||
public String answer(InvocationOnMock invocationOnMock) {
|
||||
return colNames.get((Integer) invocationOnMock.getArguments()[0]);
|
||||
|
||||
}
|
||||
|
@ -90,10 +91,10 @@ public class CassandraQueryTestUtil {
|
|||
}
|
||||
});
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ssZ");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
final Date aMonthPrior = dateFormat.parse("2016-01-03 05:00:00+0000");
|
||||
final Date testDate = dateFormat.parse("2016-02-03 05:00:00+0000");
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ssZ");
|
||||
final Date aMonthPrior = Date.from(OffsetDateTime.parse("2016-01-03 05:00:00+0000", dateTimeFormatter).toInstant());
|
||||
final Date testDate = Date.from(OffsetDateTime.parse("2016-02-03 05:00:00+0000", dateTimeFormatter).toInstant());
|
||||
|
||||
List<Row> rows = Arrays.asList(
|
||||
createRow("user1", "Joe", "Smith", Sets.newHashSet("jsmith@notareal.com"),
|
||||
Arrays.asList("New York, NY", "Santa Clara, CA"),
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.nifi.processors.cassandra;
|
|||
import com.datastax.driver.core.Cluster;
|
||||
import com.datastax.driver.core.Configuration;
|
||||
import com.datastax.driver.core.ConsistencyLevel;
|
||||
import com.datastax.driver.core.EndPoint;
|
||||
import com.datastax.driver.core.Metadata;
|
||||
import com.datastax.driver.core.ResultSet;
|
||||
import com.datastax.driver.core.ResultSetFuture;
|
||||
|
@ -42,13 +41,12 @@ import javax.net.ssl.SSLContext;
|
|||
import java.io.ByteArrayOutputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
|
@ -62,14 +60,13 @@ import static org.mockito.Mockito.doReturn;
|
|||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
|
||||
public class QueryCassandraTest {
|
||||
|
||||
private TestRunner testRunner;
|
||||
private MockQueryCassandra processor;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
public void setUp() {
|
||||
processor = new MockQueryCassandra();
|
||||
testRunner = TestRunners.newTestRunner(processor);
|
||||
}
|
||||
|
@ -113,7 +110,7 @@ public class QueryCassandraTest {
|
|||
testRunner.clearTransferState();
|
||||
|
||||
// Test exceptions
|
||||
processor.setExceptionToThrow(new NoHostAvailableException(new HashMap<EndPoint, Throwable>()));
|
||||
processor.setExceptionToThrow(new NoHostAvailableException(new HashMap<>()));
|
||||
testRunner.run(1, true, true);
|
||||
testRunner.assertAllFlowFilesTransferred(QueryCassandra.REL_RETRY, 1);
|
||||
testRunner.clearTransferState();
|
||||
|
@ -282,7 +279,7 @@ public class QueryCassandraTest {
|
|||
setUpStandardProcessorConfig();
|
||||
|
||||
// Test exceptions
|
||||
processor.setExceptionToThrow(new NoHostAvailableException(new HashMap<EndPoint, Throwable>()));
|
||||
processor.setExceptionToThrow(new NoHostAvailableException(new HashMap<>()));
|
||||
testRunner.enqueue("".getBytes());
|
||||
testRunner.run(1, true, true);
|
||||
testRunner.assertTransferCount(QueryCassandra.REL_RETRY, 1);
|
||||
|
@ -323,8 +320,6 @@ public class QueryCassandraTest {
|
|||
testRunner.assertTransferCount(QueryCassandra.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
// --
|
||||
|
||||
@Test
|
||||
public void testCreateSchemaOneColumn() throws Exception {
|
||||
ResultSet rs = CassandraQueryTestUtil.createMockResultSetOneColumn();
|
||||
|
@ -464,8 +459,7 @@ public class QueryCassandraTest {
|
|||
ResultSet rs = CassandraQueryTestUtil.createMockDateResultSet();
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
||||
DateFormat df = new SimpleDateFormat(QueryCassandra.TIMESTAMP_FORMAT_PATTERN.getDefaultValue());
|
||||
df.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(QueryCassandra.TIMESTAMP_FORMAT_PATTERN.getDefaultValue());
|
||||
|
||||
long numberOfRows = QueryCassandra.convertToJsonStream(Optional.of(testRunner.getProcessContext()), rs, 0, baos,
|
||||
StandardCharsets.UTF_8, 0, null);
|
||||
|
@ -473,7 +467,7 @@ public class QueryCassandraTest {
|
|||
|
||||
Map<String, List<Map<String, String>>> map = new ObjectMapper().readValue(baos.toByteArray(), HashMap.class);
|
||||
String date = map.get("results").get(0).get("date");
|
||||
assertEquals(df.format(CassandraQueryTestUtil.TEST_DATE), date);
|
||||
assertEquals(formatter.format(CassandraQueryTestUtil.TEST_DATE.toInstant().atOffset(ZoneOffset.UTC)), date);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -484,15 +478,14 @@ public class QueryCassandraTest {
|
|||
|
||||
final String customDateFormat = "yyyy-MM-dd HH:mm:ss.SSSZ";
|
||||
context.setProperty(QueryCassandra.TIMESTAMP_FORMAT_PATTERN, customDateFormat);
|
||||
DateFormat df = new SimpleDateFormat(customDateFormat);
|
||||
df.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(customDateFormat);
|
||||
|
||||
long numberOfRows = QueryCassandra.convertToJsonStream(Optional.of(context), rs, 0, baos, StandardCharsets.UTF_8, 0, null);
|
||||
assertEquals(1, numberOfRows);
|
||||
|
||||
Map<String, List<Map<String, String>>> map = new ObjectMapper().readValue(baos.toByteArray(), HashMap.class);
|
||||
String date = map.get("results").get(0).get("date");
|
||||
assertEquals(df.format(CassandraQueryTestUtil.TEST_DATE), date);
|
||||
assertEquals(formatter.format(CassandraQueryTestUtil.TEST_DATE.toInstant().atOffset(ZoneOffset.UTC)), date);
|
||||
}
|
||||
|
||||
private void setUpStandardProcessorConfig() {
|
||||
|
|
|
@ -55,13 +55,14 @@ import org.apache.nifi.serialization.RecordReader;
|
|||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
import org.apache.nifi.serialization.RecordSetWriter;
|
||||
import org.apache.nifi.serialization.RecordSetWriterFactory;
|
||||
import org.apache.nifi.serialization.SimpleDateFormatValidator;
|
||||
import org.apache.nifi.serialization.DateTimeFormatValidator;
|
||||
import org.apache.nifi.serialization.record.DataType;
|
||||
import org.apache.nifi.serialization.record.PushBackRecordSet;
|
||||
import org.apache.nifi.serialization.record.Record;
|
||||
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.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
@ -307,7 +308,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
+ "If specified, the value must match the Java Simple Date Format (for example, MM/dd/yyyy for a two-digit month, followed by "
|
||||
+ "a two-digit day, followed by a four-digit year, all separated by '/' characters, as in 01/25/2017).")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
|
||||
.addValidator(new SimpleDateFormatValidator())
|
||||
.addValidator(new DateTimeFormatValidator())
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
|
@ -319,7 +320,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
+ "If specified, the value must match the Java Simple Date Format (for example, HH:mm:ss for a two-digit hour in 24-hour format, followed by "
|
||||
+ "a two-digit minute, followed by a two-digit second, all separated by ':' characters, as in 18:04:15).")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
|
||||
.addValidator(new SimpleDateFormatValidator())
|
||||
.addValidator(new DateTimeFormatValidator())
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
|
@ -332,7 +333,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
+ "a two-digit day, followed by a four-digit year, all separated by '/' characters; and then followed by a two-digit hour in 24-hour format, followed by "
|
||||
+ "a two-digit minute, followed by a two-digit second, all separated by ':' characters, as in 01/25/2017 18:04:15).")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
|
||||
.addValidator(new SimpleDateFormatValidator())
|
||||
.addValidator(new DateTimeFormatValidator())
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
|
@ -622,7 +623,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
if (format != null) {
|
||||
final Object formattedValue = coerceStringToLong(
|
||||
recordField.getFieldName(),
|
||||
DataTypeUtils.toString(value, () -> DataTypeUtils.getDateFormat(format))
|
||||
StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class).convertField(value, Optional.of(format), recordField.getFieldName())
|
||||
);
|
||||
contentMap.put(recordField.getFieldName(), formattedValue);
|
||||
}
|
||||
|
@ -717,7 +718,7 @@ public class PutElasticsearchRecord extends AbstractPutElasticsearch {
|
|||
final String format = determineDateFormat(chosenDataType.getFieldType());
|
||||
returnValue = coerceStringToLong(
|
||||
fieldName,
|
||||
DataTypeUtils.toString(coercedValue, () -> DataTypeUtils.getDateFormat(format))
|
||||
StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class).convertField(coercedValue, Optional.ofNullable(format), path.getPath())
|
||||
);
|
||||
break;
|
||||
case LONG:
|
||||
|
|
|
@ -22,24 +22,20 @@ import org.apache.nifi.processors.evtx.parser.ChunkHeader;
|
|||
import org.apache.nifi.processors.evtx.parser.bxml.BxmlNode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.TimeZone;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
|
||||
/**
|
||||
* Node containing a windows file time
|
||||
*/
|
||||
public class FiletimeTypeNode extends VariantTypeNode {
|
||||
protected static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
|
||||
private final String value;
|
||||
|
||||
public FiletimeTypeNode(BinaryReader binaryReader, ChunkHeader chunkHeader, BxmlNode parent, int length) throws IOException {
|
||||
super(binaryReader, chunkHeader, parent, length);
|
||||
value = getFormat().format(binaryReader.readFileTime());
|
||||
}
|
||||
|
||||
public static final SimpleDateFormat getFormat() {
|
||||
SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
simpleDateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
return simpleDateFormat;
|
||||
value = DATE_TIME_FORMATTER.format(binaryReader.readFileTime().toInstant().atOffset(ZoneOffset.UTC));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,36 +22,31 @@ import org.apache.nifi.processors.evtx.parser.ChunkHeader;
|
|||
import org.apache.nifi.processors.evtx.parser.bxml.BxmlNode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Calendar;
|
||||
import java.util.TimeZone;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
|
||||
/**
|
||||
* Node containing a system timestamp
|
||||
*/
|
||||
public class SystemtimeTypeNode extends VariantTypeNode {
|
||||
public static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
|
||||
private final String value;
|
||||
|
||||
public SystemtimeTypeNode(BinaryReader binaryReader, ChunkHeader chunkHeader, BxmlNode parent, int length) throws IOException {
|
||||
super(binaryReader, chunkHeader, parent, length);
|
||||
int year = binaryReader.readWord();
|
||||
int month = binaryReader.readWord();
|
||||
final int monthOfYear = month + 1;
|
||||
int dayOfWeek = binaryReader.readWord();
|
||||
int day = binaryReader.readWord();
|
||||
int hour = binaryReader.readWord();
|
||||
int minute = binaryReader.readWord();
|
||||
int second = binaryReader.readWord();
|
||||
int millisecond = binaryReader.readWord();
|
||||
Calendar calendar = Calendar.getInstance();
|
||||
calendar.set(year, month, day, hour, minute, second);
|
||||
calendar.set(Calendar.MILLISECOND, millisecond);
|
||||
value = getFormat().format(calendar.getTime());
|
||||
}
|
||||
|
||||
public static final SimpleDateFormat getFormat() {
|
||||
SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
simpleDateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
return simpleDateFormat;
|
||||
final int nanosecond = millisecond * 1000000;
|
||||
final LocalDateTime localDateTime = LocalDateTime.of(year, monthOfYear, day, hour, minute, second, nanosecond);
|
||||
value = FORMATTER.format(localDateTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,8 @@ import org.apache.nifi.processors.evtx.parser.bxml.BxmlNodeTestBase;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneOffset;
|
||||
import java.util.Date;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -28,8 +30,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||
public class FiletimeTypeNodeTest extends BxmlNodeTestBase {
|
||||
@Test
|
||||
public void testFiletimeTypeNode() throws IOException {
|
||||
Date date = new Date();
|
||||
assertEquals(FiletimeTypeNode.getFormat().format(date),
|
||||
new FiletimeTypeNode(testBinaryReaderBuilder.putFileTime(date).build(), chunkHeader, parent, -1).getValue());
|
||||
final Instant instant = Instant.now();
|
||||
assertEquals(FiletimeTypeNode.DATE_TIME_FORMATTER.format(instant.atOffset(ZoneOffset.UTC)),
|
||||
new FiletimeTypeNode(testBinaryReaderBuilder.putFileTime(Date.from(instant)).build(), chunkHeader, parent, -1).getValue());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,9 @@ import org.apache.nifi.processors.evtx.parser.bxml.BxmlNodeTestBase;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.util.Calendar;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -28,8 +31,11 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||
public class SystemtimeTypeNodeTest extends BxmlNodeTestBase {
|
||||
@Test
|
||||
public void testSystemtimeTypeNode() throws IOException {
|
||||
Calendar calendar = Calendar.getInstance();
|
||||
assertEquals(SystemtimeTypeNode.getFormat().format(calendar.getTime()),
|
||||
final LocalDateTime localDateTime = LocalDateTime.now();
|
||||
final Instant instant = localDateTime.atZone(ZoneId.systemDefault()).toInstant();
|
||||
final Calendar calendar = Calendar.getInstance();
|
||||
calendar.setTimeInMillis(instant.toEpochMilli());
|
||||
assertEquals(SystemtimeTypeNode.FORMATTER.format(localDateTime),
|
||||
new SystemtimeTypeNode(testBinaryReaderBuilder.putSystemtime(calendar).build(), chunkHeader, parent, -1).getValue());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,7 +60,6 @@ import java.sql.Time;
|
|||
import java.sql.Timestamp;
|
||||
import java.sql.Types;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
|
@ -126,6 +125,9 @@ public class JdbcCommon {
|
|||
public static final String MIME_TYPE_AVRO_BINARY = "application/avro-binary";
|
||||
public static final String MASKED_LOG_VALUE = "MASKED VALUE";
|
||||
|
||||
private static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss.SSS");
|
||||
private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
|
||||
public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream, boolean convertNames) throws SQLException, IOException {
|
||||
return convertToAvroStream(rs, outStream, null, null, convertNames);
|
||||
}
|
||||
|
@ -751,7 +753,7 @@ public class JdbcCommon {
|
|||
*/
|
||||
public static void setParameter(final PreparedStatement stmt, final int parameterIndex, final String parameterValue, final int jdbcType,
|
||||
final String valueFormat)
|
||||
throws SQLException, ParseException, UnsupportedEncodingException {
|
||||
throws SQLException, UnsupportedEncodingException, ParseException {
|
||||
if (parameterValue == null) {
|
||||
stmt.setNull(parameterIndex, jdbcType);
|
||||
} else {
|
||||
|
@ -789,13 +791,11 @@ public class JdbcCommon {
|
|||
java.sql.Date date;
|
||||
|
||||
if (valueFormat.equals("")) {
|
||||
if(LONG_PATTERN.matcher(parameterValue).matches()){
|
||||
if (LONG_PATTERN.matcher(parameterValue).matches()){
|
||||
date = new java.sql.Date(Long.parseLong(parameterValue));
|
||||
}else {
|
||||
String dateFormatString = "yyyy-MM-dd";
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat(dateFormatString);
|
||||
java.util.Date parsedDate = dateFormat.parse(parameterValue);
|
||||
date = new java.sql.Date(parsedDate.getTime());
|
||||
} else {
|
||||
final LocalDate localDate = LocalDate.parse(parameterValue);
|
||||
date = java.sql.Date.valueOf(localDate);
|
||||
}
|
||||
} else {
|
||||
final DateTimeFormatter dtFormatter = getDateTimeFormatter(valueFormat);
|
||||
|
@ -812,10 +812,8 @@ public class JdbcCommon {
|
|||
if (LONG_PATTERN.matcher(parameterValue).matches()) {
|
||||
time = new Time(Long.parseLong(parameterValue));
|
||||
} else {
|
||||
String timeFormatString = "HH:mm:ss.SSS";
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat(timeFormatString);
|
||||
java.util.Date parsedDate = dateFormat.parse(parameterValue);
|
||||
time = new Time(parsedDate.getTime());
|
||||
final LocalTime localTime = LocalTime.parse(parameterValue, TIME_FORMATTER);
|
||||
time = Time.valueOf(localTime);
|
||||
}
|
||||
} else {
|
||||
final DateTimeFormatter dtFormatter = getDateTimeFormatter(valueFormat);
|
||||
|
@ -833,12 +831,11 @@ public class JdbcCommon {
|
|||
// Backwards compatibility note: Format was unsupported for a timestamp field.
|
||||
if (valueFormat.equals("")) {
|
||||
long lTimestamp = 0L;
|
||||
if(LONG_PATTERN.matcher(parameterValue).matches()){
|
||||
if (LONG_PATTERN.matcher(parameterValue).matches()){
|
||||
lTimestamp = Long.parseLong(parameterValue);
|
||||
} else {
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
java.util.Date parsedDate = dateFormat.parse(parameterValue);
|
||||
lTimestamp = parsedDate.getTime();
|
||||
final LocalDateTime localDateTime = LocalDateTime.parse(parameterValue, TIMESTAMP_FORMATTER);
|
||||
lTimestamp = Timestamp.valueOf(localDateTime).getTime();
|
||||
}
|
||||
ts = new Timestamp(lTimestamp);
|
||||
} else {
|
||||
|
|
|
@ -43,7 +43,7 @@ import java.sql.Timestamp;
|
|||
import java.sql.Types;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZoneOffset;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
|
@ -137,7 +137,7 @@ public class TestJdbcCommon {
|
|||
private static final String DERBY_LOG_PROPERTY = "derby.stream.error.file";
|
||||
|
||||
@Test
|
||||
public void testCreateSchema() throws ClassNotFoundException, SQLException {
|
||||
public void testCreateSchema() throws SQLException {
|
||||
final Statement st = con.createStatement();
|
||||
st.executeUpdate("insert into restaurants values (1, 'Irifunes', 'San Mateo')");
|
||||
st.executeUpdate("insert into restaurants values (2, 'Estradas', 'Daly City')");
|
||||
|
@ -161,7 +161,7 @@ public class TestJdbcCommon {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCreateSchemaNoColumns() throws ClassNotFoundException, SQLException {
|
||||
public void testCreateSchemaNoColumns() throws SQLException {
|
||||
|
||||
final ResultSet resultSet = mock(ResultSet.class);
|
||||
final ResultSetMetaData resultSetMetaData = mock(ResultSetMetaData.class);
|
||||
|
@ -180,7 +180,7 @@ public class TestJdbcCommon {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCreateSchemaNoTableName() throws ClassNotFoundException, SQLException {
|
||||
public void testCreateSchemaNoTableName() throws SQLException {
|
||||
|
||||
final ResultSet resultSet = mock(ResultSet.class);
|
||||
final ResultSetMetaData resultSetMetaData = mock(ResultSetMetaData.class);
|
||||
|
@ -224,7 +224,7 @@ public class TestJdbcCommon {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testConvertToBytes() throws ClassNotFoundException, SQLException, IOException {
|
||||
public void testConvertToBytes() throws SQLException, IOException {
|
||||
final Statement st = con.createStatement();
|
||||
st.executeUpdate("insert into restaurants values (1, 'Irifunes', 'San Mateo')");
|
||||
st.executeUpdate("insert into restaurants values (2, 'Estradas', 'Daly City')");
|
||||
|
@ -285,14 +285,13 @@ public class TestJdbcCommon {
|
|||
try {
|
||||
JdbcCommon.createSchema(rs);
|
||||
} catch (final IllegalArgumentException | SQLException sqle) {
|
||||
sqle.printStackTrace();
|
||||
fail("Failed when using type " + field.getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSignedIntShouldBeInt() throws SQLException, IllegalArgumentException, IllegalAccessException {
|
||||
public void testSignedIntShouldBeInt() throws SQLException, IllegalArgumentException {
|
||||
final ResultSetMetaData metadata = mock(ResultSetMetaData.class);
|
||||
when(metadata.getColumnCount()).thenReturn(1);
|
||||
when(metadata.getColumnType(1)).thenReturn(Types.INTEGER);
|
||||
|
@ -326,7 +325,7 @@ public class TestJdbcCommon {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testUnsignedIntShouldBeLong() throws SQLException, IllegalArgumentException, IllegalAccessException {
|
||||
public void testUnsignedIntShouldBeLong() throws SQLException, IllegalArgumentException {
|
||||
final ResultSetMetaData metadata = mock(ResultSetMetaData.class);
|
||||
when(metadata.getColumnCount()).thenReturn(1);
|
||||
when(metadata.getColumnType(1)).thenReturn(Types.INTEGER);
|
||||
|
@ -361,7 +360,7 @@ public class TestJdbcCommon {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testMediumUnsignedIntShouldBeInt() throws SQLException, IllegalArgumentException, IllegalAccessException {
|
||||
public void testMediumUnsignedIntShouldBeInt() throws SQLException, IllegalArgumentException {
|
||||
final ResultSetMetaData metadata = mock(ResultSetMetaData.class);
|
||||
when(metadata.getColumnCount()).thenReturn(1);
|
||||
when(metadata.getColumnType(1)).thenReturn(Types.INTEGER);
|
||||
|
@ -396,7 +395,7 @@ public class TestJdbcCommon {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testInt9ShouldBeLong() throws SQLException, IllegalArgumentException, IllegalAccessException {
|
||||
public void testInt9ShouldBeLong() throws SQLException, IllegalArgumentException {
|
||||
final ResultSetMetaData metadata = mock(ResultSetMetaData.class);
|
||||
when(metadata.getColumnCount()).thenReturn(1);
|
||||
when(metadata.getColumnType(1)).thenReturn(Types.INTEGER);
|
||||
|
@ -430,7 +429,6 @@ public class TestJdbcCommon {
|
|||
assertTrue(foundNullSchema);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testConvertToAvroStreamForBigDecimal() throws SQLException, IOException {
|
||||
final BigDecimal bigDecimal = new BigDecimal(12345D);
|
||||
|
@ -777,7 +775,7 @@ public class TestJdbcCommon {
|
|||
},
|
||||
(record, time) -> {
|
||||
int millisSinceMidnight = (int) record.get("time");
|
||||
LocalTime localTime = Instant.ofEpochMilli(millisSinceMidnight).atZone(ZoneId.systemDefault()).toLocalTime();
|
||||
LocalTime localTime = Instant.ofEpochMilli(millisSinceMidnight).atOffset(ZoneOffset.UTC).toLocalTime();
|
||||
Time actual = Time.valueOf(localTime);
|
||||
LOGGER.debug("comparing times, expecting '{}', actual '{}'", time, actual);
|
||||
assertEquals(time, actual);
|
||||
|
|
|
@ -24,7 +24,11 @@ import org.junit.jupiter.api.extension.TestWatcher;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -45,7 +49,7 @@ public class ListProcessorTestWatcher implements TestWatcher, BeforeEachCallback
|
|||
T provide();
|
||||
}
|
||||
|
||||
private final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS");
|
||||
private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ISO_DATE_TIME;
|
||||
private final Provider<Map<String, String>> stateMapProvider;
|
||||
private final Provider<List<ListableEntity>> entitiesProvider;
|
||||
private final Provider<List<FlowFile>> successFlowFilesProvider;
|
||||
|
@ -68,36 +72,39 @@ public class ListProcessorTestWatcher implements TestWatcher, BeforeEachCallback
|
|||
|
||||
private void dumpState(Consumer<String> d, final Map<String, String> state, final List<ListableEntity> entities, final List<FlowFile> flowFiles, final long start) {
|
||||
|
||||
final long nTime = System.currentTimeMillis();
|
||||
final OffsetDateTime nTime = OffsetDateTime.now();
|
||||
log(d, "--------------------------------------------------------------------");
|
||||
log(d, "%-19s %-13s %-23s %s", "", "timestamp", "date from timestamp", "t0 delta");
|
||||
log(d, "%-19s %-13s %-23s %s", "-------------------", "-------------", "-----------------------", "--------");
|
||||
log(d, "%-19s = %13d %s %8d", "started at", start, dateFormat.format(start), 0);
|
||||
log(d, "%-19s = %13d %s %8d", "current time", nTime, dateFormat.format(nTime), 0);
|
||||
log(d, "%-19s = %13d %s %8d", "started at", start, dateTimeFormatter.format(Instant.ofEpochMilli(start).atZone(ZoneId.systemDefault())), 0);
|
||||
log(d, "%-19s = %13d %s %8d", "current time", nTime.toInstant().toEpochMilli(), dateTimeFormatter.format(nTime), 0);
|
||||
log(d, "---- processor state -----------------------------------------------");
|
||||
if (state.containsKey("processed.timestamp")) {
|
||||
final long pTime = Long.parseLong(state.get("processed.timestamp"));
|
||||
log(d, "%19s = %13d %s %8d", "processed.timestamp", pTime, dateFormat.format(pTime), pTime - nTime);
|
||||
final OffsetDateTime processedTime = OffsetDateTime.ofInstant(Instant.ofEpochMilli(pTime), ZoneOffset.UTC);
|
||||
log(d, "%19s = %13d %s %8d", "processed.timestamp", pTime, dateTimeFormatter.format(processedTime), pTime - nTime.toInstant().toEpochMilli());
|
||||
} else {
|
||||
log(d, "%19s = na", "processed.timestamp");
|
||||
}
|
||||
if (state.containsKey("listing.timestamp")) {
|
||||
final long lTime = Long.parseLong(state.get("listing.timestamp"));
|
||||
log(d, "%19s = %13d %s %8d", "listing.timestamp", lTime, dateFormat.format(lTime), lTime - nTime);
|
||||
log(d, "%19s = %13d %s %8d", "listing.timestamp", lTime, dateTimeFormatter.format(Instant.ofEpochMilli(lTime).atZone(ZoneId.systemDefault())), lTime - nTime.toInstant().toEpochMilli());
|
||||
} else {
|
||||
log(d, "%19s = na", "listing.timestamp");
|
||||
}
|
||||
log(d, "---- input folder contents -----------------------------------------");
|
||||
entities.sort(Comparator.comparing(ListableEntity::getIdentifier));
|
||||
for (ListableEntity entity : entities) {
|
||||
log(d, "%19s = %12d %s %8d", entity.getIdentifier(), entity.getTimestamp(), dateFormat.format(entity.getTimestamp()), entity.getTimestamp() - nTime);
|
||||
final OffsetDateTime timestamp = OffsetDateTime.ofInstant(Instant.ofEpochMilli(entity.getTimestamp()), ZoneId.systemDefault());
|
||||
log(d, "%19s = %12d %s %8d", entity.getIdentifier(), entity.getTimestamp(), dateTimeFormatter.format(timestamp), entity.getTimestamp() - nTime.toInstant().toEpochMilli());
|
||||
}
|
||||
log(d, "---- output flowfiles ----------------------------------------------");
|
||||
final Map<String, Long> fileTimes = entities.stream().collect(Collectors.toMap(ListableEntity::getIdentifier, ListableEntity::getTimestamp));
|
||||
for (FlowFile ff : flowFiles) {
|
||||
String fName = ff.getAttribute(CoreAttributes.FILENAME.key());
|
||||
Long fTime = fileTimes.get(fName);
|
||||
log(d, "%19s = %13d %s %8d", fName, fTime, dateFormat.format(fTime), fTime - nTime);
|
||||
final OffsetDateTime timestamp = OffsetDateTime.ofInstant(Instant.ofEpochMilli(fTime), ZoneId.systemDefault());
|
||||
log(d, "%19s = %13d %s %8d", fName, fTime, dateTimeFormatter.format(timestamp), fTime - nTime.toInstant().toEpochMilli());
|
||||
}
|
||||
log(d, "REL_SUCCESS count = " + flowFiles.size());
|
||||
log(d, "--------------------------------------------------------------------");
|
||||
|
|
|
@ -17,23 +17,23 @@
|
|||
|
||||
package org.apache.nifi.avro;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.RoundingMode;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Blob;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -68,6 +68,8 @@ import org.apache.nifi.serialization.record.RecordFieldType;
|
|||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.serialization.record.SchemaIdentifier;
|
||||
import org.apache.nifi.serialization.record.StandardSchemaIdentifier;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.type.DecimalDataType;
|
||||
|
@ -91,6 +93,7 @@ public class AvroTypeUtil {
|
|||
private static final String LOGICAL_TYPE_DECIMAL = "decimal";
|
||||
private static final String LOGICAL_TYPE_UUID = "uuid";
|
||||
|
||||
private static final long ONE_THOUSAND_MILLISECONDS = 1000;
|
||||
|
||||
public static Schema extractAvroSchema(final RecordSchema recordSchema) {
|
||||
if (recordSchema == null) {
|
||||
|
@ -592,11 +595,11 @@ public class AvroTypeUtil {
|
|||
return new ImmutablePair<>(fieldName, field);
|
||||
}
|
||||
|
||||
public static GenericRecord createAvroRecord(final Record record, final Schema avroSchema) throws IOException {
|
||||
public static GenericRecord createAvroRecord(final Record record, final Schema avroSchema) {
|
||||
return createAvroRecord(record, avroSchema, StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
public static GenericRecord createAvroRecord(final Record record, final Schema avroSchema, final Charset charset) throws IOException {
|
||||
public static GenericRecord createAvroRecord(final Record record, final Schema avroSchema, final Charset charset) {
|
||||
final GenericRecord rec = new GenericData.Record(avroSchema);
|
||||
final RecordSchema recordSchema = record.getSchema();
|
||||
|
||||
|
@ -692,8 +695,9 @@ public class AvroTypeUtil {
|
|||
|
||||
private static Long getLongFromTimestamp(final Object rawValue, final Schema fieldSchema, final String fieldName) {
|
||||
final String format = AvroTypeUtil.determineDataType(fieldSchema).getFormat();
|
||||
Timestamp t = DataTypeUtils.toTimestamp(rawValue, () -> DataTypeUtils.getDateFormat(format), fieldName);
|
||||
return t.getTime();
|
||||
final FieldConverter<Object, Timestamp> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class);
|
||||
final Timestamp timestamp = converter.convertField(rawValue, Optional.ofNullable(format), fieldName);
|
||||
return timestamp.getTime();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -709,17 +713,15 @@ public class AvroTypeUtil {
|
|||
return DataTypeUtils.toInteger(rawValue, fieldName);
|
||||
}
|
||||
|
||||
|
||||
if (LOGICAL_TYPE_DATE.equals(logicalType.getName())) {
|
||||
final String format = AvroTypeUtil.determineDataType(fieldSchema).getFormat();
|
||||
final LocalDate localDate = DataTypeUtils.toLocalDate(rawValue, () -> DataTypeUtils.getDateTimeFormatter(format, ZoneId.systemDefault()), fieldName);
|
||||
final FieldConverter<Object, LocalDate> fieldConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
final LocalDate localDate = fieldConverter.convertField(rawValue, Optional.ofNullable(format), fieldName);
|
||||
return (int) localDate.toEpochDay();
|
||||
} else if (LOGICAL_TYPE_TIME_MILLIS.equals(logicalType.getName())) {
|
||||
final String format = AvroTypeUtil.determineDataType(fieldSchema).getFormat();
|
||||
final Time time = DataTypeUtils.toTime(rawValue, () -> DataTypeUtils.getDateFormat(format), fieldName);
|
||||
final Date date = new Date(time.getTime());
|
||||
final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant());
|
||||
final long millisSinceMidnight = duration.toMillis();
|
||||
return (int) millisSinceMidnight;
|
||||
return getLogicalTimeMillis(rawValue, format, fieldName);
|
||||
}
|
||||
|
||||
return DataTypeUtils.toInteger(rawValue, fieldName);
|
||||
|
@ -731,14 +733,15 @@ public class AvroTypeUtil {
|
|||
}
|
||||
|
||||
if (LOGICAL_TYPE_TIME_MICROS.equals(logicalType.getName())) {
|
||||
final long longValue = getLongFromTimestamp(rawValue, fieldSchema, fieldName);
|
||||
final Date date = new Date(longValue);
|
||||
final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant());
|
||||
return duration.toMillis() * 1000L;
|
||||
final long epochMilli = getLongFromTimestamp(rawValue, fieldSchema, fieldName);
|
||||
final ZonedDateTime zonedDateTime = Instant.ofEpochMilli(epochMilli).atZone(ZoneId.systemDefault());
|
||||
final ZonedDateTime midnight = zonedDateTime.truncatedTo(ChronoUnit.DAYS);
|
||||
final Duration duration = Duration.between(midnight, zonedDateTime);
|
||||
return duration.toMillis() * ONE_THOUSAND_MILLISECONDS;
|
||||
} else if (LOGICAL_TYPE_TIMESTAMP_MILLIS.equals(logicalType.getName())) {
|
||||
return getLongFromTimestamp(rawValue, fieldSchema, fieldName);
|
||||
} else if (LOGICAL_TYPE_TIMESTAMP_MICROS.equals(logicalType.getName())) {
|
||||
return getLongFromTimestamp(rawValue, fieldSchema, fieldName) * 1000L;
|
||||
return getLongFromTimestamp(rawValue, fieldSchema, fieldName) * ONE_THOUSAND_MILLISECONDS;
|
||||
}
|
||||
|
||||
return DataTypeUtils.toLong(rawValue, fieldName);
|
||||
|
@ -1174,4 +1177,13 @@ public class AvroTypeUtil {
|
|||
return value;
|
||||
}
|
||||
|
||||
private static int getLogicalTimeMillis(final Object value, final String format, final String fieldName) {
|
||||
final FieldConverter<Object, LocalTime> fieldConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalTime.class);
|
||||
final LocalTime localTime = fieldConverter.convertField(value, Optional.ofNullable(format), fieldName);
|
||||
|
||||
final LocalTime midnightLocalTime = localTime.truncatedTo(ChronoUnit.DAYS);
|
||||
final Duration duration = Duration.between(midnightLocalTime, localTime);
|
||||
final long millisSinceMidnight = duration.toMillis();
|
||||
return (int) millisSinceMidnight;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,14 +44,13 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.BiPredicate;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public abstract class AbstractJsonRowRecordReader implements RecordReader {
|
||||
public static final String DEFAULT_MAX_STRING_LENGTH = "20 MB";
|
||||
|
@ -80,9 +79,9 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
|
|||
.build();
|
||||
|
||||
private final ComponentLog logger;
|
||||
private final Supplier<DateFormat> lazyDateFormat;
|
||||
private final Supplier<DateFormat> lazyTimeFormat;
|
||||
private final Supplier<DateFormat> lazyTimestampFormat;
|
||||
private final String dateFormat;
|
||||
private final String timeFormat;
|
||||
private final String timestampFormat;
|
||||
|
||||
private boolean firstObjectConsumed = false;
|
||||
private JsonParser jsonParser;
|
||||
|
@ -94,13 +93,9 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
|
|||
private AbstractJsonRowRecordReader(final ComponentLog logger, final String dateFormat, final String timeFormat, final String timestampFormat) {
|
||||
this.logger = logger;
|
||||
|
||||
final DateFormat df = dateFormat == null ? null : DataTypeUtils.getDateFormat(dateFormat);
|
||||
final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
|
||||
final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
|
||||
|
||||
lazyDateFormat = () -> df;
|
||||
lazyTimeFormat = () -> tf;
|
||||
lazyTimestampFormat = () -> tsf;
|
||||
this.dateFormat = dateFormat;
|
||||
this.timeFormat = timeFormat;
|
||||
this.timestampFormat = timestampFormat;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -170,16 +165,16 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
|
|||
}
|
||||
}
|
||||
|
||||
protected Supplier<DateFormat> getLazyDateFormat() {
|
||||
return lazyDateFormat;
|
||||
protected Optional<String> getDateFormat() {
|
||||
return Optional.ofNullable(dateFormat);
|
||||
}
|
||||
|
||||
protected Supplier<DateFormat> getLazyTimeFormat() {
|
||||
return lazyTimeFormat;
|
||||
protected Optional<String> getTimeFormat() {
|
||||
return Optional.ofNullable(timeFormat);
|
||||
}
|
||||
|
||||
protected Supplier<DateFormat> getLazyTimestampFormat() {
|
||||
return lazyTimestampFormat;
|
||||
protected Optional<String> getTimestampFormat() {
|
||||
return Optional.ofNullable(timestampFormat);
|
||||
}
|
||||
|
||||
|
||||
|
@ -238,7 +233,7 @@ public abstract class AbstractJsonRowRecordReader implements RecordReader {
|
|||
case TIME:
|
||||
case TIMESTAMP:
|
||||
try {
|
||||
return DataTypeUtils.convertType(textValue, dataType, lazyDateFormat, lazyTimeFormat, lazyTimestampFormat, fieldName);
|
||||
return DataTypeUtils.convertType(textValue, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName);
|
||||
} catch (final Exception e) {
|
||||
return textValue;
|
||||
}
|
||||
|
|
|
@ -196,7 +196,7 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
|
|||
case UUID:
|
||||
case TIMESTAMP:
|
||||
try {
|
||||
return DataTypeUtils.convertType(value, dataType, getLazyDateFormat(), getLazyTimeFormat(), getLazyTimestampFormat(), fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, getDateFormat(), getTimeFormat(), getTimestampFormat(), fieldName);
|
||||
} catch (final Exception e) {
|
||||
return value;
|
||||
}
|
||||
|
@ -249,7 +249,7 @@ public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
|
|||
|
||||
return new MapRecord(childSchema, coercedValues);
|
||||
} else {
|
||||
return DataTypeUtils.convertType(value, dataType, getLazyDateFormat(), getLazyTimeFormat(), getLazyTimestampFormat(), fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, getDateFormat(), getTimeFormat(), getTimestampFormat(), fieldName);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -217,7 +217,7 @@ public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
|
|||
case UUID:
|
||||
case TIMESTAMP: {
|
||||
final Object rawValue = getRawNodeValue(fieldNode, fieldName);
|
||||
return DataTypeUtils.convertType(rawValue, desiredType, getLazyDateFormat(), getLazyTimeFormat(), getLazyTimestampFormat(), fieldName);
|
||||
return DataTypeUtils.convertType(rawValue, desiredType, getDateFormat(), getTimeFormat(), getTimestampFormat(), fieldName);
|
||||
}
|
||||
case MAP: {
|
||||
final DataType valueType = ((MapDataType) desiredType).getValueType();
|
||||
|
|
|
@ -34,6 +34,8 @@ 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.serialization.record.SerializedForm;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.type.MapDataType;
|
||||
|
@ -43,24 +45,23 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
|||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.math.BigInteger;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSetWriter, RawRecordWriter {
|
||||
private static final FieldConverter<Object, String> STRING_FIELD_CONVERTER = StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class);
|
||||
|
||||
private final ComponentLog logger;
|
||||
private final SchemaAccessWriter schemaAccess;
|
||||
private final RecordSchema recordSchema;
|
||||
private final JsonGenerator generator;
|
||||
private final NullSuppression nullSuppression;
|
||||
private final OutputGrouping outputGrouping;
|
||||
private final Supplier<DateFormat> LAZY_DATE_FORMAT;
|
||||
private final Supplier<DateFormat> LAZY_TIME_FORMAT;
|
||||
private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
|
||||
private String mimeType = "application/json";
|
||||
private final String dateFormat;
|
||||
private final String timeFormat;
|
||||
private final String timestampFormat;
|
||||
private final String mimeType;
|
||||
private final boolean prettyPrint;
|
||||
|
||||
private static final ObjectMapper objectMapper = new ObjectMapper();
|
||||
|
@ -82,14 +83,9 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
this.outputGrouping = outputGrouping;
|
||||
this.mimeType = mimeType;
|
||||
|
||||
// Use DateFormat with default TimeZone to avoid unexpected conversion of year-month-day
|
||||
final DateFormat df = dateFormat == null ? null : new SimpleDateFormat(dateFormat);
|
||||
final DateFormat tf = timeFormat == null ? null : DataTypeUtils.getDateFormat(timeFormat);
|
||||
final DateFormat tsf = timestampFormat == null ? null : DataTypeUtils.getDateFormat(timestampFormat);
|
||||
|
||||
LAZY_DATE_FORMAT = () -> df;
|
||||
LAZY_TIME_FORMAT = () -> tf;
|
||||
LAZY_TIMESTAMP_FORMAT = () -> tsf;
|
||||
this.dateFormat = dateFormat;
|
||||
this.timeFormat = timeFormat;
|
||||
this.timestampFormat = timestampFormat;
|
||||
|
||||
final JsonFactory factory = new JsonFactory();
|
||||
factory.setCodec(objectMapper);
|
||||
|
@ -251,8 +247,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
return;
|
||||
}
|
||||
|
||||
if (value instanceof Record) {
|
||||
final Record record = (Record) value;
|
||||
if (value instanceof Record record) {
|
||||
writeRecord(record, record.getSchema(), generator, JsonGenerator::writeStartObject, JsonGenerator::writeEndObject, false);
|
||||
return;
|
||||
}
|
||||
|
@ -272,8 +267,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
return;
|
||||
}
|
||||
|
||||
if (value instanceof Object[]) {
|
||||
final Object[] values = (Object[]) value;
|
||||
if (value instanceof Object[] values) {
|
||||
generator.writeStartArray();
|
||||
for (final Object element : values) {
|
||||
writeRawValue(generator, element, fieldName);
|
||||
|
@ -283,17 +277,17 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
}
|
||||
|
||||
if (value instanceof java.sql.Time) {
|
||||
final Object formatted = format((java.sql.Time) value, LAZY_TIME_FORMAT);
|
||||
final Object formatted = STRING_FIELD_CONVERTER.convertField(value, Optional.ofNullable(timeFormat), fieldName);
|
||||
generator.writeObject(formatted);
|
||||
return;
|
||||
}
|
||||
if (value instanceof java.sql.Date) {
|
||||
final Object formatted = format((java.sql.Date) value, LAZY_DATE_FORMAT);
|
||||
final Object formatted = STRING_FIELD_CONVERTER.convertField(value, Optional.ofNullable(dateFormat), fieldName);
|
||||
generator.writeObject(formatted);
|
||||
return;
|
||||
}
|
||||
if (value instanceof java.util.Date) {
|
||||
final Object formatted = format((java.util.Date) value, LAZY_TIMESTAMP_FORMAT);
|
||||
final Object formatted = STRING_FIELD_CONVERTER.convertField(value, Optional.ofNullable(timestampFormat), fieldName);
|
||||
generator.writeObject(formatted);
|
||||
return;
|
||||
}
|
||||
|
@ -301,22 +295,6 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
generator.writeObject(value);
|
||||
}
|
||||
|
||||
private Object format(final java.util.Date value, final Supplier<DateFormat> formatSupplier) {
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (formatSupplier == null) {
|
||||
return value.getTime();
|
||||
}
|
||||
final DateFormat format = formatSupplier.get();
|
||||
if (format == null) {
|
||||
return value.getTime();
|
||||
}
|
||||
|
||||
return format.format(value);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void writeValue(final JsonGenerator generator, final Object value, final String fieldName, final DataType dataType) throws IOException {
|
||||
if (value == null) {
|
||||
|
@ -326,12 +304,14 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
|
||||
final DataType chosenDataType = dataType.getFieldType() == RecordFieldType.CHOICE ? DataTypeUtils.chooseDataType(value, (ChoiceDataType) dataType) : dataType;
|
||||
if (chosenDataType == null) {
|
||||
logger.debug("Could not find a suitable field type in the CHOICE for field {} and value {}; will use null value", new Object[] {fieldName, value});
|
||||
logger.debug("Could not find a suitable field type in the CHOICE for field {} and value {}; will use null value", fieldName, value);
|
||||
generator.writeNull();
|
||||
return;
|
||||
}
|
||||
|
||||
final Object coercedValue = DataTypeUtils.convertType(value, chosenDataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
|
||||
final Object coercedValue = DataTypeUtils.convertType(
|
||||
value, chosenDataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName
|
||||
);
|
||||
if (coercedValue == null) {
|
||||
generator.writeNull();
|
||||
return;
|
||||
|
@ -339,7 +319,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
|
||||
switch (chosenDataType.getFieldType()) {
|
||||
case DATE: {
|
||||
final String stringValue = DataTypeUtils.toString(coercedValue, LAZY_DATE_FORMAT);
|
||||
final String stringValue = STRING_FIELD_CONVERTER.convertField(coercedValue, Optional.ofNullable(dateFormat), fieldName);
|
||||
if (DataTypeUtils.isLongTypeCompatible(stringValue)) {
|
||||
generator.writeNumber(DataTypeUtils.toLong(coercedValue, fieldName));
|
||||
} else {
|
||||
|
@ -348,7 +328,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
break;
|
||||
}
|
||||
case TIME: {
|
||||
final String stringValue = DataTypeUtils.toString(coercedValue, LAZY_TIME_FORMAT);
|
||||
final String stringValue = STRING_FIELD_CONVERTER.convertField(coercedValue, Optional.ofNullable(timeFormat), fieldName);
|
||||
if (DataTypeUtils.isLongTypeCompatible(stringValue)) {
|
||||
generator.writeNumber(DataTypeUtils.toLong(coercedValue, fieldName));
|
||||
} else {
|
||||
|
@ -357,7 +337,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
break;
|
||||
}
|
||||
case TIMESTAMP: {
|
||||
final String stringValue = DataTypeUtils.toString(coercedValue, LAZY_TIMESTAMP_FORMAT);
|
||||
final String stringValue = STRING_FIELD_CONVERTER.convertField(coercedValue, Optional.ofNullable(timestampFormat), fieldName);
|
||||
if (DataTypeUtils.isLongTypeCompatible(stringValue)) {
|
||||
generator.writeNumber(DataTypeUtils.toLong(coercedValue, fieldName));
|
||||
} else {
|
||||
|
@ -389,7 +369,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
break;
|
||||
case BIGINT:
|
||||
if (coercedValue instanceof Long) {
|
||||
generator.writeNumber(((Long) coercedValue).longValue());
|
||||
generator.writeNumber((Long) coercedValue);
|
||||
} else {
|
||||
generator.writeNumber((BigInteger) coercedValue);
|
||||
}
|
||||
|
@ -428,8 +408,7 @@ public class WriteJsonResult extends AbstractRecordSetWriter implements RecordSe
|
|||
}
|
||||
case ARRAY:
|
||||
default:
|
||||
if (coercedValue instanceof Object[]) {
|
||||
final Object[] values = (Object[]) coercedValue;
|
||||
if (coercedValue instanceof Object[] values) {
|
||||
final ArrayDataType arrayDataType = (ArrayDataType) chosenDataType;
|
||||
final DataType elementType = arrayDataType.getElementType();
|
||||
writeArray(values, fieldName, generator, elementType);
|
||||
|
|
|
@ -17,18 +17,18 @@
|
|||
|
||||
package org.apache.nifi.serialization;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.components.Validator;
|
||||
|
||||
public class SimpleDateFormatValidator implements Validator {
|
||||
import java.time.format.DateTimeFormatter;
|
||||
|
||||
public class DateTimeFormatValidator implements Validator {
|
||||
|
||||
@Override
|
||||
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
|
||||
try {
|
||||
new SimpleDateFormat(input);
|
||||
DateTimeFormatter.ofPattern(input);
|
||||
} catch (final Exception e) {
|
||||
return new ValidationResult.Builder()
|
||||
.input(input)
|
||||
|
@ -44,5 +44,4 @@ public class SimpleDateFormatValidator implements Validator {
|
|||
.valid(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
}
|
|
@ -28,7 +28,7 @@ public class DateTimeUtils {
|
|||
+ "If specified, the value must match the Java Simple Date Format (for example, MM/dd/yyyy for a two-digit month, followed by "
|
||||
+ "a two-digit day, followed by a four-digit year, all separated by '/' characters, as in 01/01/2017).")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.addValidator(new SimpleDateFormatValidator())
|
||||
.addValidator(new DateTimeFormatValidator())
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
|
@ -39,7 +39,7 @@ public class DateTimeUtils {
|
|||
+ "If specified, the value must match the Java Simple Date Format (for example, HH:mm:ss for a two-digit hour in 24-hour format, followed by "
|
||||
+ "a two-digit minute, followed by a two-digit second, all separated by ':' characters, as in 18:04:15).")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.addValidator(new SimpleDateFormatValidator())
|
||||
.addValidator(new DateTimeFormatValidator())
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
|
@ -51,7 +51,7 @@ public class DateTimeUtils {
|
|||
+ "a two-digit day, followed by a four-digit year, all separated by '/' characters; and then followed by a two-digit hour in 24-hour format, followed by "
|
||||
+ "a two-digit minute, followed by a two-digit second, all separated by ':' characters, as in 01/01/2017 18:04:15).")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
|
||||
.addValidator(new SimpleDateFormatValidator())
|
||||
.addValidator(new DateTimeFormatValidator())
|
||||
.required(false)
|
||||
.build();
|
||||
}
|
||||
|
|
|
@ -35,9 +35,6 @@ import java.math.BigInteger;
|
|||
import java.sql.Date;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.text.DateFormat;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -48,7 +45,6 @@ import java.util.LinkedHashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -75,7 +71,7 @@ public class TestStandardSchemaValidator {
|
|||
));
|
||||
|
||||
@Test
|
||||
public void testValidateCorrectSimpleTypesStrictValidation() throws ParseException {
|
||||
public void testValidateCorrectSimpleTypesStrictValidation() {
|
||||
final List<RecordField> fields = new ArrayList<>();
|
||||
for (final RecordFieldType fieldType : RecordFieldType.values()) {
|
||||
if (fieldType == RecordFieldType.CHOICE) {
|
||||
|
@ -97,9 +93,7 @@ public class TestStandardSchemaValidator {
|
|||
}
|
||||
}
|
||||
|
||||
final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
|
||||
df.setTimeZone(TimeZone.getTimeZone("gmt"));
|
||||
final long time = df.parse("2017/01/01 17:00:00.000").getTime();
|
||||
final long time = 1483290000000L;
|
||||
|
||||
final Map<String, Object> intMap = new LinkedHashMap<>();
|
||||
intMap.put("height", 48);
|
||||
|
@ -229,7 +223,7 @@ public class TestStandardSchemaValidator {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testValidateWrongButCoerceableType() throws ParseException {
|
||||
public void testValidateWrongButCoerceableType() {
|
||||
final List<RecordField> fields = new ArrayList<>();
|
||||
fields.add(new RecordField("id", RecordFieldType.INT.getDataType()));
|
||||
final RecordSchema schema = new SimpleRecordSchema(fields);
|
||||
|
|
|
@ -16,10 +16,10 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api.dto.util;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Date;
|
||||
import java.util.Locale;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import jakarta.xml.bind.annotation.adapters.XmlAdapter;
|
||||
|
||||
|
@ -28,20 +28,19 @@ import jakarta.xml.bind.annotation.adapters.XmlAdapter;
|
|||
*/
|
||||
public class DateTimeAdapter extends XmlAdapter<String, Date> {
|
||||
|
||||
public static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss z";
|
||||
private static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss z";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(DEFAULT_DATE_TIME_FORMAT);
|
||||
|
||||
@Override
|
||||
public String marshal(Date date) throws Exception {
|
||||
final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
|
||||
formatter.setTimeZone(TimeZone.getDefault());
|
||||
return formatter.format(date);
|
||||
final ZonedDateTime zonedDateTime = date.toInstant().atZone(ZoneId.systemDefault());
|
||||
return DATE_TIME_FORMATTER.format(zonedDateTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Date unmarshal(String date) throws Exception {
|
||||
final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
|
||||
parser.setTimeZone(TimeZone.getDefault());
|
||||
return parser.parse(date);
|
||||
final ZonedDateTime zonedDateTime = ZonedDateTime.parse(date, DATE_TIME_FORMATTER);
|
||||
return Date.from(zonedDateTime.toInstant());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,10 +16,11 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api.dto.util;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Date;
|
||||
import java.util.Locale;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import jakarta.xml.bind.annotation.adapters.XmlAdapter;
|
||||
|
||||
|
@ -28,20 +29,18 @@ import jakarta.xml.bind.annotation.adapters.XmlAdapter;
|
|||
*/
|
||||
public class TimestampAdapter extends XmlAdapter<String, Date> {
|
||||
|
||||
public static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss.SSS z";
|
||||
private static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss.SSS z";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(DEFAULT_DATE_TIME_FORMAT).withZone(ZoneId.systemDefault());
|
||||
|
||||
@Override
|
||||
public String marshal(Date date) throws Exception {
|
||||
final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
|
||||
formatter.setTimeZone(TimeZone.getDefault());
|
||||
return formatter.format(date);
|
||||
return DATE_TIME_FORMATTER.format(date.toInstant());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Date unmarshal(String date) throws Exception {
|
||||
final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT, Locale.US);
|
||||
parser.setTimeZone(TimeZone.getDefault());
|
||||
return parser.parse(date);
|
||||
final ZonedDateTime dateTime = LocalDateTime.parse(date, DATE_TIME_FORMATTER).atZone(ZoneId.systemDefault());
|
||||
return Date.from(dateTime.toInstant());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -63,12 +63,11 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -302,8 +301,7 @@ public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvide
|
|||
final AuthorizationsHolder holder = authorizationsHolder.get();
|
||||
final Authorizations authorizations = holder.getAuthorizations();
|
||||
|
||||
final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
|
||||
final String timestamp = dateFormat.format(new Date());
|
||||
final String timestamp = DateTimeFormatter.ofPattern("yyyy-MM-dd-HH-mm-ss").format(OffsetDateTime.now());
|
||||
|
||||
final File backupFile = new File(authorizationsFile.getParentFile(), authorizationsFile.getName() + "." + timestamp);
|
||||
logger.info("Writing backup of Policies to {}", backupFile.getAbsolutePath());
|
||||
|
|
|
@ -59,12 +59,11 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -492,8 +491,7 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
|
|||
final UserGroupHolder holder = userGroupHolder.get();
|
||||
final Tenants tenants = holder.getTenants();
|
||||
|
||||
final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
|
||||
final String timestamp = dateFormat.format(new Date());
|
||||
final String timestamp = DateTimeFormatter.ofPattern("yyyy-MM-dd-HH-mm-ss").format(OffsetDateTime.now());
|
||||
final File destinationFile = new File(tenantsFile.getParentFile(), tenantsFile.getName() + "." + timestamp);
|
||||
logger.info("Writing backup of Users & Groups to {}", destinationFile.getAbsolutePath());
|
||||
|
||||
|
|
|
@ -19,22 +19,17 @@ package org.apache.nifi.cluster.coordination.http.endpoints;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.text.DateFormat;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Locale;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestStatusHistoryEndpointMerger {
|
||||
@Test
|
||||
public void testNormalizedStatusSnapshotDate() throws ParseException {
|
||||
final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:SS.SSS", Locale.US);
|
||||
final Date date1 = df.parse("2014/01/01 00:00:00.000");
|
||||
final Date date2 = df.parse("2014/01/01 00:04:59.999");
|
||||
final Date date3 = df.parse("2014/01/01 00:05:00.000");
|
||||
final Date date4 = df.parse("2014/01/01 00:05:00.001");
|
||||
public void testNormalizedStatusSnapshotDate() {
|
||||
final Date date1 = new Date(1388538000000L);
|
||||
final Date date2 = new Date(1388538299999L);
|
||||
final Date date3 = new Date(1388538300000L);
|
||||
final Date date4 = new Date(1388538300001L);
|
||||
|
||||
final Date normalized1 = StatusHistoryEndpointMerger.normalizeStatusSnapshotDate(date1, 300000);
|
||||
assertEquals(date1, normalized1);
|
||||
|
|
|
@ -18,9 +18,7 @@ package org.apache.nifi.controller;
|
|||
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
|
||||
import java.text.DateFormat;
|
||||
import java.text.NumberFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
@ -32,7 +30,6 @@ public class StandardGarbageCollectionEvent implements GarbageCollectionEvent {
|
|||
private final long startTime;
|
||||
private final long endTime;
|
||||
private final List<GarbageCollectionHeapSize> heapSizes;
|
||||
private final DateFormat dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.SSS");
|
||||
|
||||
public StandardGarbageCollectionEvent(final String gcName, final String action, final String cause, final long startTime, final long endTime, final List<GarbageCollectionHeapSize> heapSizes) {
|
||||
this.gcName = gcName;
|
||||
|
@ -84,8 +81,8 @@ public class StandardGarbageCollectionEvent implements GarbageCollectionEvent {
|
|||
sb.append("GarbageCollectionEvent[collectorName=").append(gcName)
|
||||
.append(", action=").append(action)
|
||||
.append(", cause=").append(cause)
|
||||
.append(", startTime=").append(dateFormat.format(new Date(startTime)))
|
||||
.append(", endTime=").append(dateFormat.format(new Date(endTime)))
|
||||
.append(", startTime=").append(new Date(startTime).toInstant())
|
||||
.append(", endTime=").append(new Date(endTime).toInstant())
|
||||
.append(", duration=").append(NumberFormat.getInstance().format(endTime - startTime))
|
||||
.append(" ms, heap sizes={");
|
||||
|
||||
|
|
|
@ -31,11 +31,8 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -102,7 +99,6 @@ public class ClusterProtocolHeartbeater implements Heartbeater {
|
|||
final long sendNanos = System.nanoTime() - sendStart;
|
||||
final long sendMillis = TimeUnit.NANOSECONDS.toMillis(sendNanos);
|
||||
|
||||
final DateFormat dateFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS", Locale.US);
|
||||
final String flowElectionMessage = responseMessage.getFlowElectionMessage();
|
||||
final String formattedElectionMessage = flowElectionMessage == null ? "" : "; " + flowElectionMessage;
|
||||
|
||||
|
@ -110,9 +106,9 @@ public class ClusterProtocolHeartbeater implements Heartbeater {
|
|||
|
||||
logger.info("Heartbeat created at {} and sent to {} at {}; determining Cluster Coordinator took {} millis; DNS lookup for coordinator took {} millis; connecting to coordinator took {} " +
|
||||
"millis; sending heartbeat took {} millis; receiving first byte from response took {} millis; receiving full response took {} millis; total time was {} millis{}",
|
||||
dateFormatter.format(new Date(heartbeatMessage.getHeartbeat().getCreatedTimestamp())),
|
||||
Instant.ofEpochMilli(heartbeatMessage.getHeartbeat().getCreatedTimestamp()),
|
||||
heartbeatAddress,
|
||||
dateFormatter.format(new Date()),
|
||||
Instant.now(),
|
||||
TimeUnit.NANOSECONDS.toMillis(findCoordinatorNanos),
|
||||
timingDetails.getDnsLookupMillis(),
|
||||
timingDetails.getConnectMillis(),
|
||||
|
|
|
@ -32,7 +32,8 @@ import java.nio.file.SimpleFileVisitor;
|
|||
import java.nio.file.StandardOpenOption;
|
||||
import java.nio.file.attribute.BasicFileAttributes;
|
||||
import java.nio.file.attribute.FileTime;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Date;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -160,13 +161,13 @@ public class TestFlowConfigurationArchiveManager {
|
|||
|
||||
// Create old archive files. Altering file name and last modified date to simulate existing files.
|
||||
final long now = System.currentTimeMillis();
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat("HHmmss");
|
||||
final DateTimeFormatter dateFormat = DateTimeFormatter.ofPattern("HHmmss");
|
||||
|
||||
final FlowConfigurationArchiveManager archiveManager = createArchiveManager(null,null, null);
|
||||
|
||||
for (int i = oldArchives.length; i > 0; i--) {
|
||||
final Date date = new Date(now - (intervalMillis * i));
|
||||
final String hhmmss = dateFormat.format(date);
|
||||
final String hhmmss = dateFormat.format(date.toInstant().atZone(ZoneId.systemDefault()));
|
||||
|
||||
final File archiveFile = archiveManager.archive(flowXmlFile);
|
||||
final String renamedArchiveName = archiveFile.getName().replaceFirst("T[\\d]{6}", "T" + hhmmss);
|
||||
|
|
|
@ -145,7 +145,8 @@ import jakarta.ws.rs.core.Response;
|
|||
import jakarta.ws.rs.core.StreamingOutput;
|
||||
|
||||
import java.text.Collator;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -800,8 +801,8 @@ public class FlowResource extends ApplicationResource {
|
|||
? serviceFacade.getVersionedReportingTaskSnapshot() :
|
||||
serviceFacade.getVersionedReportingTaskSnapshot(reportingTaskId);
|
||||
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat(VERSIONED_REPORTING_TASK_SNAPSHOT_DATE_FORMAT);
|
||||
final String filename = VERSIONED_REPORTING_TASK_SNAPSHOT_FILENAME_PATTERN.formatted(dateFormat.format(new Date()));
|
||||
final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(VERSIONED_REPORTING_TASK_SNAPSHOT_DATE_FORMAT);
|
||||
final String filename = VERSIONED_REPORTING_TASK_SNAPSHOT_FILENAME_PATTERN.formatted(formatter.format(OffsetDateTime.now()));
|
||||
return generateOkResponse(snapshot).header(HttpHeaders.CONTENT_DISPOSITION, String.format("attachment; filename=\"%s\"", filename)).build();
|
||||
}
|
||||
|
||||
|
|
|
@ -16,10 +16,11 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api.request;
|
||||
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.Date;
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Class for parsing integer parameters and providing a user friendly error message.
|
||||
|
@ -27,14 +28,15 @@ import java.util.Locale;
|
|||
public class DateTimeParameter {
|
||||
|
||||
private static final String DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(DATE_TIME_FORMAT);
|
||||
private static final String INVALID_INTEGER_MESSAGE = "Unable to parse '%s' as a date/time. Expected format '%s'.";
|
||||
|
||||
private Date dateTimeValue;
|
||||
private final Date dateTimeValue;
|
||||
|
||||
public DateTimeParameter(String rawDateTime) {
|
||||
try {
|
||||
dateTimeValue = parse(rawDateTime);
|
||||
} catch (ParseException pe) {
|
||||
} catch (DateTimeParseException pe) {
|
||||
throw new IllegalArgumentException(String.format(INVALID_INTEGER_MESSAGE,
|
||||
rawDateTime, DATE_TIME_FORMAT));
|
||||
}
|
||||
|
@ -44,19 +46,8 @@ public class DateTimeParameter {
|
|||
return dateTimeValue;
|
||||
}
|
||||
|
||||
public static String format(final Date date) {
|
||||
SimpleDateFormat parser = new SimpleDateFormat(DATE_TIME_FORMAT, Locale.US);
|
||||
parser.setLenient(false);
|
||||
return parser.format(date);
|
||||
}
|
||||
|
||||
public static String format(final DateTimeParameter param) {
|
||||
return format(param.getDateTime());
|
||||
}
|
||||
|
||||
public static Date parse(final String str) throws ParseException {
|
||||
SimpleDateFormat parser = new SimpleDateFormat(DATE_TIME_FORMAT, Locale.US);
|
||||
parser.setLenient(false);
|
||||
return parser.parse(str);
|
||||
private static Date parse(final String str) {
|
||||
final LocalDateTime localDateTime = LocalDateTime.parse(str, DATE_TIME_FORMATTER);
|
||||
return Date.from(localDateTime.atZone(ZoneId.systemDefault()).toInstant());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,23 +46,25 @@ import org.apache.nifi.serialization.record.Record;
|
|||
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.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.MapDataType;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class NiFiRecordSerDe extends AbstractSerDe {
|
||||
|
||||
|
@ -83,7 +85,7 @@ public class NiFiRecordSerDe extends AbstractSerDe {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void initialize(Configuration conf, Properties tbl) throws SerDeException {
|
||||
public void initialize(Configuration conf, Properties tbl) {
|
||||
List<TypeInfo> columnTypes;
|
||||
StructTypeInfo rowTypeInfo;
|
||||
|
||||
|
@ -108,14 +110,10 @@ public class NiFiRecordSerDe extends AbstractSerDe {
|
|||
columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
|
||||
}
|
||||
|
||||
log.debug("columns: {}, {}", new Object[]{columnNameProperty, columnNames});
|
||||
log.debug("types: {}, {} ", new Object[]{columnTypeProperty, columnTypes});
|
||||
|
||||
assert (columnNames.size() == columnTypes.size());
|
||||
|
||||
rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
|
||||
schema = rowTypeInfo;
|
||||
log.debug("schema : {}", new Object[]{schema});
|
||||
cachedObjectInspector = (StandardStructObjectInspector) TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(rowTypeInfo);
|
||||
tsParser = new TimestampParser(HiveStringUtils.splitAndUnEscape(tbl.getProperty(serdeConstants.TIMESTAMP_FORMATS)));
|
||||
stats = new SerDeStats();
|
||||
|
@ -127,7 +125,7 @@ public class NiFiRecordSerDe extends AbstractSerDe {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException {
|
||||
public Writable serialize(Object o, ObjectInspector objectInspector) {
|
||||
throw new UnsupportedOperationException("This SerDe only supports deserialization");
|
||||
}
|
||||
|
||||
|
@ -221,18 +219,16 @@ public class NiFiRecordSerDe extends AbstractSerDe {
|
|||
val = AvroTypeUtil.convertByteArray(array).array();
|
||||
break;
|
||||
case DATE:
|
||||
Date d = DataTypeUtils.toDate(fieldValue, () -> DataTypeUtils.getDateFormat(fieldDataType.getFormat()), fieldName);
|
||||
org.apache.hadoop.hive.common.type.Date hiveDate = new org.apache.hadoop.hive.common.type.Date();
|
||||
hiveDate.setTimeInMillis(d.getTime());
|
||||
val = hiveDate;
|
||||
final FieldConverter<Object, LocalDate> dateConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
final LocalDate localDate = dateConverter.convertField(fieldValue, Optional.ofNullable(fieldDataType.getFormat()), fieldName);
|
||||
final long epochDay = localDate.toEpochDay();
|
||||
val = org.apache.hadoop.hive.common.type.Date.ofEpochDay((int) epochDay);
|
||||
break;
|
||||
// ORC doesn't currently handle TIMESTAMPLOCALTZ
|
||||
case TIMESTAMP:
|
||||
Timestamp ts = DataTypeUtils.toTimestamp(fieldValue, () -> DataTypeUtils.getDateFormat(fieldDataType.getFormat()), fieldName);
|
||||
// Convert to Hive's Timestamp type
|
||||
org.apache.hadoop.hive.common.type.Timestamp hivetimestamp = new org.apache.hadoop.hive.common.type.Timestamp();
|
||||
hivetimestamp.setTimeInMillis(ts.getTime(), ts.getNanos());
|
||||
val = hivetimestamp;
|
||||
final FieldConverter<Object, Timestamp> timestampConverter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class);
|
||||
final Timestamp timestamp = timestampConverter.convertField(fieldValue, Optional.ofNullable(fieldDataType.getFormat()), fieldName);
|
||||
val = org.apache.hadoop.hive.common.type.Timestamp.valueOf(timestamp.toString());
|
||||
break;
|
||||
case DECIMAL:
|
||||
if(fieldValue instanceof BigDecimal){
|
||||
|
@ -299,16 +295,16 @@ public class NiFiRecordSerDe extends AbstractSerDe {
|
|||
String normalizedFieldName = fieldName.toLowerCase();
|
||||
|
||||
// Normalize struct field names and search for the specified (normalized) field name
|
||||
int fpos = typeInfo.getAllStructFieldNames().stream().map((s) -> s == null ? null : s.toLowerCase()).collect(Collectors.toList()).indexOf(normalizedFieldName);
|
||||
int fpos = typeInfo.getAllStructFieldNames().stream().map((s) -> s == null ? null : s.toLowerCase()).toList().indexOf(normalizedFieldName);
|
||||
if (fpos == -1) {
|
||||
Matcher m = INTERNAL_PATTERN.matcher(fieldName);
|
||||
fpos = m.matches() ? Integer.parseInt(m.group(1)) : -1;
|
||||
|
||||
log.debug("NPE finding position for field [{}] in schema [{}],"
|
||||
+ " attempting to check if it is an internal column name like _col0", new Object[]{fieldName, typeInfo});
|
||||
+ " attempting to check if it is an internal column name like _col0", fieldName, typeInfo);
|
||||
if (fpos == -1) {
|
||||
// unknown field, we return. We'll continue from the next field onwards. Log at debug level because partition columns will be "unknown fields"
|
||||
log.debug("Field {} is not found in the target table, ignoring...", new Object[]{field.getFieldName()});
|
||||
log.debug("Field {} is not found in the target table, ignoring...", field.getFieldName());
|
||||
return;
|
||||
}
|
||||
// If we get past this, then the column name did match the hive pattern for an internal
|
||||
|
@ -317,7 +313,7 @@ public class NiFiRecordSerDe extends AbstractSerDe {
|
|||
// if we find it.
|
||||
if (!fieldName.equalsIgnoreCase(HiveConf.getColumnInternalName(fpos))) {
|
||||
log.error("Hive internal column name {} and position "
|
||||
+ "encoding {} for the column name are at odds", new Object[]{fieldName, fpos});
|
||||
+ "encoding {} for the column name are at odds", fieldName, fpos);
|
||||
throw new SerDeException("Hive internal column name (" + fieldName
|
||||
+ ") and position encoding (" + fpos
|
||||
+ ") for the column name are at odds");
|
||||
|
|
|
@ -72,7 +72,11 @@ public class TestNiFiRecordSerDe {
|
|||
)
|
||||
);
|
||||
|
||||
long currentTimeMillis = System.currentTimeMillis();
|
||||
final String localDate = "2015-01-01";
|
||||
Date date = Date.valueOf(localDate);
|
||||
|
||||
final String localDateTime = "2015-01-01 12:30:45";
|
||||
Timestamp ts = Timestamp.valueOf(localDateTime);
|
||||
|
||||
HashMap<String, Object> input = new HashMap<String, Object>() {{
|
||||
put("bytec", (byte) 2);
|
||||
|
@ -85,16 +89,11 @@ public class TestNiFiRecordSerDe {
|
|||
put("stringc", "test");
|
||||
put("varcharc", "test2");
|
||||
put("charc", 'c');
|
||||
put("datec", new java.sql.Date(currentTimeMillis));
|
||||
put("timestampc", new java.sql.Timestamp(currentTimeMillis));
|
||||
put("datec", java.sql.Date.valueOf(localDate));
|
||||
put("timestampc", java.sql.Timestamp.valueOf(localDateTime));
|
||||
put("decimalc", 0.45);
|
||||
}};
|
||||
|
||||
Date date = new Date();
|
||||
date.setTimeInMillis(currentTimeMillis);
|
||||
Timestamp ts = new Timestamp();
|
||||
ts.setTimeInMillis(currentTimeMillis);
|
||||
|
||||
List<Object> expected = Arrays.asList(
|
||||
Byte.valueOf("2"),
|
||||
Short.valueOf("45"),
|
||||
|
@ -176,11 +175,12 @@ public class TestNiFiRecordSerDe {
|
|||
|
||||
@Test
|
||||
public void testSimpleArray() throws SerDeException{
|
||||
long now = System.currentTimeMillis();
|
||||
Date hiveDate = new Date();
|
||||
hiveDate.setTimeInMillis(now);
|
||||
Timestamp hiveTs = new Timestamp();
|
||||
hiveTs.setTimeInMillis(now);
|
||||
|
||||
final String localDate = "2015-01-01";
|
||||
Date hiveDate = Date.valueOf(localDate);
|
||||
|
||||
final String localDateTime = "2015-01-01 12:30:45";
|
||||
Timestamp hiveTs = Timestamp.valueOf(localDateTime);
|
||||
|
||||
testSimpleArray("tinyint", RecordFieldType.BYTE.getDataType(), new Byte[] { 5, 29 },
|
||||
new Byte[] { 5, 29 });
|
||||
|
@ -202,9 +202,9 @@ public class TestNiFiRecordSerDe {
|
|||
new Object[] { "niko", "fiti", "sema" });
|
||||
testSimpleArray("char(1)", RecordFieldType.CHAR.getDataType(), new Object[] { 'a', 'b', 'c' },
|
||||
new Object[] { "a", "b", "c"});
|
||||
testSimpleArray("date", RecordFieldType.DATE.getDataType(), new Object[] { new java.sql.Date(now)},
|
||||
testSimpleArray("date", RecordFieldType.DATE.getDataType(), new Object[] { java.sql.Date.valueOf(localDate)},
|
||||
new Object[] { hiveDate });
|
||||
testSimpleArray("timestamp", RecordFieldType.TIMESTAMP.getDataType(), new Object[] { new java.sql.Timestamp(now)},
|
||||
testSimpleArray("timestamp", RecordFieldType.TIMESTAMP.getDataType(), new Object[] { java.sql.Timestamp.valueOf(localDateTime)},
|
||||
new Object[] { hiveTs });
|
||||
testSimpleArray("decimal(10,2)", RecordFieldType.DOUBLE.getDataType(), new Object[] { 3.45, 1.25 },
|
||||
new Object[] { HiveDecimal.create(3.45), HiveDecimal.create(1.25)});
|
||||
|
@ -280,14 +280,15 @@ public class TestNiFiRecordSerDe {
|
|||
testSimpleMap("string", "string", RecordFieldType.STRING.getDataType(), createMap("form", "ni", "aje"), objectMap(createMap("form", "ni", "aje")));
|
||||
testSimpleMap("string", "varchar(20)", RecordFieldType.STRING.getDataType(), createMap("niko", "kiza"), objectMap(createMap("niko", "kiza")));
|
||||
testSimpleMap("string", "char(1)", RecordFieldType.CHAR.getDataType(), createMap('a', 'b', 'c'), objectMap(createMap("a", "b", "c")));
|
||||
long now = System.currentTimeMillis();
|
||||
Date hiveDate = new Date();
|
||||
hiveDate.setTimeInMillis(now);
|
||||
Timestamp hiveTs = new Timestamp();
|
||||
hiveTs.setTimeInMillis(now);
|
||||
|
||||
testSimpleMap("string", "date", RecordFieldType.DATE.getDataType(), createMap(new java.sql.Date(now)), objectMap(createMap(hiveDate)));
|
||||
testSimpleMap("string", "timestamp", RecordFieldType.TIMESTAMP.getDataType(), createMap(new java.sql.Timestamp(now)), objectMap(createMap(hiveTs)));
|
||||
final String localDate = "2015-01-01";
|
||||
Date hiveDate = Date.valueOf(localDate);
|
||||
|
||||
final String localDateTime = "2015-01-01 12:30:45";
|
||||
Timestamp hiveTs = Timestamp.valueOf(localDateTime);
|
||||
|
||||
testSimpleMap("string", "date", RecordFieldType.DATE.getDataType(), createMap(java.sql.Date.valueOf(localDate)), objectMap(createMap(hiveDate)));
|
||||
testSimpleMap("string", "timestamp", RecordFieldType.TIMESTAMP.getDataType(), createMap(java.sql.Timestamp.valueOf(localDateTime)), objectMap(createMap(hiveTs)));
|
||||
testSimpleMap("string", "decimal(10,2)", RecordFieldType.DOUBLE.getDataType(), createMap(45.6, 2345.5), objectMap(createMap(
|
||||
HiveDecimal.create(45.6), HiveDecimal.create(2345.5)
|
||||
)));
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.nifi.flowfile.FlowFile;
|
|||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processors.hadoop.exception.FailureException;
|
||||
import org.apache.nifi.processors.hadoop.record.HDFSRecordWriter;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
|
@ -68,8 +67,6 @@ import java.math.BigDecimal;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Date;
|
||||
import java.sql.Timestamp;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.LocalTime;
|
||||
|
@ -77,7 +74,6 @@ import java.time.temporal.ChronoField;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TimeZone;
|
||||
import java.util.function.BiFunction;
|
||||
|
||||
import static org.apache.nifi.processors.hadoop.AbstractHadoopProcessor.HADOOP_FILE_URL_ATTRIBUTE;
|
||||
|
@ -220,13 +216,13 @@ public class PutORCTest {
|
|||
public void testWriteORCWithAvroLogicalTypes() throws IOException, InitializationException {
|
||||
final String avroSchema = IOUtils.toString(new FileInputStream("src/test/resources/user_logical_types.avsc"), StandardCharsets.UTF_8);
|
||||
schema = new Schema.Parser().parse(avroSchema);
|
||||
LocalTime nowTime = LocalTime.now();
|
||||
LocalDateTime nowDateTime = LocalDateTime.now();
|
||||
LocalDate nowDate = LocalDate.now();
|
||||
LocalTime localTime = LocalTime.of(12, 30, 45);
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2024, 1, 1, 12, 30, 45);
|
||||
LocalDate localDate = LocalDate.of(2024, 1, 1);
|
||||
|
||||
final int timeMillis = nowTime.get(ChronoField.MILLI_OF_DAY);
|
||||
final Timestamp timestampMillis = Timestamp.valueOf(nowDateTime);
|
||||
final Date dt = Date.valueOf(nowDate);
|
||||
final int timeMillis = localTime.get(ChronoField.MILLI_OF_DAY);
|
||||
final Timestamp timestampMillis = Timestamp.valueOf(localDateTime);
|
||||
final Date dt = new Date(1704112245000L);
|
||||
final BigDecimal bigDecimal = new BigDecimal("92.12");
|
||||
|
||||
configure(proc, 10, (numUsers, readerFactory) -> {
|
||||
|
@ -279,9 +275,10 @@ public class PutORCTest {
|
|||
assertEquals((int) currUser, ((IntWritable) x.get(0)).get());
|
||||
assertEquals(timeMillis, ((IntWritable) x.get(1)).get());
|
||||
assertEquals(timestampMillis, ((TimestampWritableV2) x.get(2)).getTimestamp().toSqlTimestamp());
|
||||
final DateFormat noTimeOfDayDateFormat = new SimpleDateFormat("yyyy-MM-dd");
|
||||
noTimeOfDayDateFormat.setTimeZone(TimeZone.getTimeZone("gmt"));
|
||||
assertEquals(noTimeOfDayDateFormat.format(dt), ((DateWritableV2) x.get(3)).get().toString());
|
||||
|
||||
final DateWritableV2 dateWritableV2 = (DateWritableV2) x.get(3);
|
||||
final int epochDay = dateWritableV2.get().toEpochDay();
|
||||
assertEquals(localDate.toEpochDay(), epochDay);
|
||||
assertEquals(bigDecimal, ((HiveDecimalWritable) x.get(4)).getHiveDecimal().bigDecimalValue());
|
||||
return null;
|
||||
}
|
||||
|
@ -391,8 +388,7 @@ public class PutORCTest {
|
|||
public void testIOExceptionRenamingShouldRouteToRetry() throws InitializationException {
|
||||
final PutORC proc = new PutORC() {
|
||||
@Override
|
||||
protected void rename(FileSystem fileSystem, Path srcFile, Path destFile)
|
||||
throws IOException, InterruptedException, FailureException {
|
||||
protected void rename(FileSystem fileSystem, Path srcFile, Path destFile) throws IOException {
|
||||
throw new IOException("IOException renaming");
|
||||
}
|
||||
};
|
||||
|
@ -449,7 +445,7 @@ public class PutORCTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDDLQuoteTableNameSections() throws IOException, InitializationException {
|
||||
public void testDDLQuoteTableNameSections() throws InitializationException {
|
||||
configure(proc, 100);
|
||||
|
||||
final String filename = "testORCWithDefaults-" + System.currentTimeMillis();
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
package org.apache.nifi.processors.iceberg.converter;
|
||||
|
||||
import org.apache.nifi.serialization.record.DataType;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
|
@ -25,7 +27,10 @@ import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Serializable;
|
||||
import java.time.ZoneId;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.util.Optional;
|
||||
|
||||
public class ArrayElementGetter {
|
||||
|
||||
|
@ -62,10 +67,16 @@ public class ArrayElementGetter {
|
|||
elementGetter = element -> DataTypeUtils.toInteger(element, ARRAY_FIELD_NAME);
|
||||
break;
|
||||
case DATE:
|
||||
elementGetter = element -> DataTypeUtils.toLocalDate(element, () -> DataTypeUtils.getDateTimeFormatter(dataType.getFormat(), ZoneId.systemDefault()), ARRAY_FIELD_NAME);
|
||||
elementGetter = element -> {
|
||||
final FieldConverter<Object, LocalDate> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
return converter.convertField(element, Optional.ofNullable(dataType.getFormat()), ARRAY_FIELD_NAME);
|
||||
};
|
||||
break;
|
||||
case TIME:
|
||||
elementGetter = element -> DataTypeUtils.toTime(element, () -> DataTypeUtils.getDateFormat(dataType.getFormat()), ARRAY_FIELD_NAME);
|
||||
elementGetter = element -> {
|
||||
final FieldConverter<Object, Time> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Time.class);
|
||||
return converter.convertField(element, Optional.ofNullable(dataType.getFormat()), ARRAY_FIELD_NAME);
|
||||
};
|
||||
break;
|
||||
case LONG:
|
||||
elementGetter = element -> DataTypeUtils.toLong(element, ARRAY_FIELD_NAME);
|
||||
|
@ -80,7 +91,10 @@ public class ArrayElementGetter {
|
|||
elementGetter = element -> DataTypeUtils.toDouble(element, ARRAY_FIELD_NAME);
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
elementGetter = element -> DataTypeUtils.toTimestamp(element, () -> DataTypeUtils.getDateFormat(dataType.getFormat()), ARRAY_FIELD_NAME);
|
||||
elementGetter = element -> {
|
||||
final FieldConverter<Object, Timestamp> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class);
|
||||
return converter.convertField(element, Optional.ofNullable(dataType.getFormat()), ARRAY_FIELD_NAME);
|
||||
};
|
||||
break;
|
||||
case UUID:
|
||||
elementGetter = DataTypeUtils::toUUID;
|
||||
|
|
|
@ -27,16 +27,16 @@ import org.apache.nifi.serialization.record.Record;
|
|||
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.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.LocalTime;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -74,12 +74,13 @@ public class GenericDataConverters {
|
|||
case DOUBLE:
|
||||
return DataTypeUtils.toDouble(data, null);
|
||||
case DATE:
|
||||
return DataTypeUtils.toLocalDate(data, () -> DataTypeUtils.getDateTimeFormatter(sourceType.getFormat(), ZoneId.systemDefault()), null);
|
||||
final FieldConverter<Object, LocalDate> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
return converter.convertField(data, Optional.ofNullable(sourceType.getFormat()), null);
|
||||
case UUID:
|
||||
return DataTypeUtils.toUUID(data);
|
||||
case STRING:
|
||||
default:
|
||||
return DataTypeUtils.toString(data, () -> null);
|
||||
return StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class).convertField(data, Optional.empty(), null);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -94,8 +95,8 @@ public class GenericDataConverters {
|
|||
|
||||
@Override
|
||||
public LocalTime convert(Object data) {
|
||||
Time time = DataTypeUtils.toTime(data, () -> DataTypeUtils.getDateFormat(timeFormat), null);
|
||||
return time == null ? null : time.toLocalTime();
|
||||
final FieldConverter<Object, LocalTime> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalTime.class);
|
||||
return converter.convertField(data, Optional.ofNullable(timeFormat), null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -109,8 +110,8 @@ public class GenericDataConverters {
|
|||
|
||||
@Override
|
||||
public LocalDateTime convert(Object data) {
|
||||
final Timestamp convertedTimestamp = DataTypeUtils.toTimestamp(data, () -> DataTypeUtils.getDateFormat(dataType.getFormat()), null);
|
||||
return convertedTimestamp == null ? null : convertedTimestamp.toLocalDateTime();
|
||||
final FieldConverter<Object, LocalDateTime> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDateTime.class);
|
||||
return converter.convertField(data, Optional.ofNullable(dataType.getFormat()), null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -124,8 +125,8 @@ public class GenericDataConverters {
|
|||
|
||||
@Override
|
||||
public OffsetDateTime convert(Object data) {
|
||||
final Timestamp convertedTimestamp = DataTypeUtils.toTimestamp(data, () -> DataTypeUtils.getDateFormat(dataType.getFormat()), null);
|
||||
return convertedTimestamp == null ? null : OffsetDateTime.ofInstant(convertedTimestamp.toInstant(), ZoneId.of("UTC"));
|
||||
final FieldConverter<Object, OffsetDateTime> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(OffsetDateTime.class);
|
||||
return converter.convertField(data, Optional.ofNullable(dataType.getFormat()), null);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.nifi.processors.iceberg.converter;
|
|||
|
||||
import org.apache.nifi.serialization.record.DataType;
|
||||
import org.apache.nifi.serialization.record.Record;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ArrayDataType;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.type.RecordDataType;
|
||||
|
@ -27,7 +29,10 @@ import org.apache.nifi.serialization.record.util.IllegalTypeConversionException;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Serializable;
|
||||
import java.time.ZoneId;
|
||||
import java.sql.Time;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.util.Optional;
|
||||
|
||||
public class RecordFieldGetter {
|
||||
|
||||
|
@ -64,10 +69,16 @@ public class RecordFieldGetter {
|
|||
fieldGetter = record -> record.getAsInt(fieldName);
|
||||
break;
|
||||
case DATE:
|
||||
fieldGetter = record -> DataTypeUtils.toLocalDate(record.getValue(fieldName), () -> DataTypeUtils.getDateTimeFormatter(dataType.getFormat(), ZoneId.systemDefault()), fieldName);
|
||||
fieldGetter = record -> {
|
||||
final FieldConverter<Object, LocalDate> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
return converter.convertField(record.getValue(fieldName), Optional.ofNullable(dataType.getFormat()), fieldName);
|
||||
};
|
||||
break;
|
||||
case TIME:
|
||||
fieldGetter = record -> DataTypeUtils.toTime(record.getValue(fieldName), () -> DataTypeUtils.getDateFormat(dataType.getFormat()), fieldName);
|
||||
fieldGetter = record -> {
|
||||
final FieldConverter<Object, Time> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Time.class);
|
||||
return converter.convertField(record.getValue(fieldName), Optional.ofNullable(dataType.getFormat()), fieldName);
|
||||
};
|
||||
break;
|
||||
case LONG:
|
||||
fieldGetter = record -> record.getAsLong(fieldName);
|
||||
|
@ -82,7 +93,10 @@ public class RecordFieldGetter {
|
|||
fieldGetter = record -> record.getAsDouble(fieldName);
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
fieldGetter = record -> DataTypeUtils.toTimestamp(record.getValue(fieldName), () -> DataTypeUtils.getDateFormat(dataType.getFormat()), fieldName);
|
||||
fieldGetter = record -> {
|
||||
final FieldConverter<Object, Timestamp> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class);
|
||||
return converter.convertField(record.getValue(fieldName), Optional.ofNullable(dataType.getFormat()), fieldName);
|
||||
};
|
||||
break;
|
||||
case UUID:
|
||||
fieldGetter = record -> DataTypeUtils.toUUID(record.getValue(fieldName));
|
||||
|
|
|
@ -70,7 +70,6 @@ import java.time.LocalDate;
|
|||
import java.time.LocalDateTime;
|
||||
import java.time.LocalTime;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZoneOffset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
@ -93,6 +92,8 @@ import static org.junit.jupiter.api.condition.OS.WINDOWS;
|
|||
|
||||
public class TestIcebergRecordConverter {
|
||||
|
||||
private static final LocalDateTime LOCAL_DATE_TIME = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
|
||||
private OutputFile tempFile;
|
||||
|
||||
private ComponentLog logger;
|
||||
|
@ -293,9 +294,9 @@ public class TestIcebergRecordConverter {
|
|||
fields.add(new RecordField("fixed", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType())));
|
||||
fields.add(new RecordField("binary", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType())));
|
||||
fields.add(new RecordField("date", RecordFieldType.STRING.getDataType("yyyy-MM-dd")));
|
||||
fields.add(new RecordField("time", RecordFieldType.STRING.getDataType("hh:mm:ss.SSS")));
|
||||
fields.add(new RecordField("timestamp", RecordFieldType.STRING.getDataType("yyyy-MM-dd hh:mm:ss.SSSZ")));
|
||||
fields.add(new RecordField("timestampTz", RecordFieldType.STRING.getDataType("yyyy-MM-dd hh:mm:ss.SSSZ")));
|
||||
fields.add(new RecordField("time", RecordFieldType.STRING.getDataType("HH:mm:ss.SSS")));
|
||||
fields.add(new RecordField("timestamp", RecordFieldType.STRING.getDataType("yyyy-MM-dd HH:mm:ss.SSSZ")));
|
||||
fields.add(new RecordField("timestampTz", RecordFieldType.STRING.getDataType("yyyy-MM-dd HH:mm:ss.SSSZ")));
|
||||
fields.add(new RecordField("uuid", RecordFieldType.STRING.getDataType()));
|
||||
fields.add(new RecordField("choice", RecordFieldType.CHOICE.getChoiceDataType(RecordFieldType.STRING.getDataType(), RecordFieldType.INT.getDataType())));
|
||||
|
||||
|
@ -389,8 +390,7 @@ public class TestIcebergRecordConverter {
|
|||
private static Record setupPrimitivesTestRecord() {
|
||||
LocalDate localDate = LocalDate.of(2017, 4, 4);
|
||||
LocalTime localTime = LocalTime.of(14, 20, 33);
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(localDateTime, ZoneOffset.ofHours(-5));
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5));
|
||||
|
||||
Map<String, Object> values = new HashMap<>();
|
||||
values.put("string", "Test String");
|
||||
|
@ -405,7 +405,7 @@ public class TestIcebergRecordConverter {
|
|||
values.put("date", localDate);
|
||||
values.put("time", Time.valueOf(localTime));
|
||||
values.put("timestamp", Timestamp.from(offsetDateTime.toInstant()));
|
||||
values.put("timestampTz", Timestamp.valueOf(localDateTime));
|
||||
values.put("timestampTz", Timestamp.valueOf(LOCAL_DATE_TIME));
|
||||
values.put("uuid", UUID.fromString("0000-00-00-00-000000"));
|
||||
values.put("choice", "10");
|
||||
|
||||
|
@ -415,8 +415,7 @@ public class TestIcebergRecordConverter {
|
|||
private static Record setupPrimitivesTestRecordMissingFields() {
|
||||
LocalDate localDate = LocalDate.of(2017, 4, 4);
|
||||
LocalTime localTime = LocalTime.of(14, 20, 33);
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(localDateTime, ZoneOffset.ofHours(-5));
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5));
|
||||
|
||||
Map<String, Object> values = new HashMap<>();
|
||||
values.put("string", "Test String");
|
||||
|
@ -428,7 +427,7 @@ public class TestIcebergRecordConverter {
|
|||
values.put("date", localDate);
|
||||
values.put("time", Time.valueOf(localTime));
|
||||
values.put("timestamp", Timestamp.from(offsetDateTime.toInstant()));
|
||||
values.put("timestampTz", Timestamp.valueOf(localDateTime));
|
||||
values.put("timestampTz", Timestamp.valueOf(LOCAL_DATE_TIME));
|
||||
values.put("uuid", UUID.fromString("0000-00-00-00-000000"));
|
||||
values.put("choice", "10");
|
||||
|
||||
|
@ -514,8 +513,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(results.size(), 1);
|
||||
GenericRecord resultRecord = results.getFirst();
|
||||
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(localDateTime, ZoneOffset.ofHours(-5));
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5));
|
||||
|
||||
assertEquals("Test String", resultRecord.get(0, String.class));
|
||||
assertEquals(Integer.valueOf(8), resultRecord.get(1, Integer.class));
|
||||
|
@ -529,7 +527,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(9, LocalDate.class));
|
||||
assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(10, LocalTime.class));
|
||||
assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(11, OffsetDateTime.class));
|
||||
assertEquals(LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000), resultRecord.get(12, LocalDateTime.class));
|
||||
assertEquals(LOCAL_DATE_TIME, resultRecord.get(12, LocalDateTime.class));
|
||||
assertEquals(Integer.valueOf(10), resultRecord.get(14, Integer.class));
|
||||
|
||||
if (format.equals(PARQUET)) {
|
||||
|
@ -557,8 +555,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(results.size(), 1);
|
||||
GenericRecord resultRecord = results.getFirst();
|
||||
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(localDateTime, ZoneOffset.ofHours(-5));
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5));
|
||||
|
||||
assertEquals("Test String", resultRecord.get(0, String.class));
|
||||
assertNull(resultRecord.get(1, Integer.class));
|
||||
|
@ -572,7 +569,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(9, LocalDate.class));
|
||||
assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(10, LocalTime.class));
|
||||
assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(11, OffsetDateTime.class));
|
||||
assertEquals(LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000), resultRecord.get(12, LocalDateTime.class));
|
||||
assertEquals(LOCAL_DATE_TIME, resultRecord.get(12, LocalDateTime.class));
|
||||
assertEquals(Integer.valueOf(10), resultRecord.get(14, Integer.class));
|
||||
|
||||
if (format.equals(FileFormat.PARQUET)) {
|
||||
|
@ -643,8 +640,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(results.size(), 1);
|
||||
GenericRecord resultRecord = results.getFirst();
|
||||
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(localDateTime, ZoneOffset.ofHours(-5));
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5));
|
||||
|
||||
assertEquals("Test String", resultRecord.get(0, String.class));
|
||||
assertNull(resultRecord.get(1, Integer.class));
|
||||
|
@ -658,7 +654,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(9, LocalDate.class));
|
||||
assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(10, LocalTime.class));
|
||||
assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(11, OffsetDateTime.class));
|
||||
assertEquals(LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000), resultRecord.get(12, LocalDateTime.class));
|
||||
assertEquals(LOCAL_DATE_TIME, resultRecord.get(12, LocalDateTime.class));
|
||||
assertEquals(Integer.valueOf(10), resultRecord.get(14, Integer.class));
|
||||
|
||||
if (format.equals(FileFormat.PARQUET)) {
|
||||
|
@ -676,7 +672,7 @@ public class TestIcebergRecordConverter {
|
|||
@DisabledOnOs(WINDOWS)
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = FileFormat.class, names = {"AVRO", "ORC", "PARQUET"})
|
||||
public void testPrimitivesFailMissingFields(FileFormat format) throws IOException {
|
||||
public void testPrimitivesFailMissingFields(FileFormat format) {
|
||||
RecordSchema nifiSchema = getPrimitivesSchemaMissingFields();
|
||||
MockComponentLogger mockComponentLogger = new MockComponentLogger();
|
||||
|
||||
|
@ -701,9 +697,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(results.size(), 1);
|
||||
GenericRecord resultRecord = results.getFirst();
|
||||
|
||||
LocalDateTime localDateTime = LocalDateTime.of(2017, 4, 4, 14, 20, 33, 789000000);
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(localDateTime, ZoneOffset.ofHours(-5));
|
||||
LocalDateTime expectedLocalDateTimestamp = offsetDateTime.atZoneSameInstant(ZoneId.systemDefault()).toLocalDateTime();
|
||||
OffsetDateTime offsetDateTime = OffsetDateTime.of(LOCAL_DATE_TIME, ZoneOffset.ofHours(-5));
|
||||
|
||||
assertEquals("123", resultRecord.get(0, String.class));
|
||||
assertEquals(Integer.valueOf(8), resultRecord.get(1, Integer.class));
|
||||
|
@ -716,7 +710,7 @@ public class TestIcebergRecordConverter {
|
|||
assertEquals(LocalDate.of(2017, 4, 4), resultRecord.get(8, LocalDate.class));
|
||||
assertEquals(LocalTime.of(14, 20, 33), resultRecord.get(9, LocalTime.class));
|
||||
assertEquals(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), resultRecord.get(10, OffsetDateTime.class));
|
||||
assertEquals(expectedLocalDateTimestamp, resultRecord.get(11, LocalDateTime.class));
|
||||
assertEquals(LOCAL_DATE_TIME, resultRecord.get(11, LocalDateTime.class));
|
||||
assertEquals(Integer.valueOf(10), resultRecord.get(13, Integer.class));
|
||||
|
||||
assertArrayEquals(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, resultRecord.get(12, byte[].class));
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.math.BigDecimal;
|
|||
import java.sql.Date;
|
||||
import java.sql.Timestamp;
|
||||
import java.time.LocalDate;
|
||||
import java.time.ZoneId;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
@ -63,7 +62,7 @@ import org.apache.nifi.serialization.record.DataType;
|
|||
import org.apache.nifi.serialization.record.Record;
|
||||
import org.apache.nifi.serialization.record.RecordFieldType;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.ObjectTimestampFieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.DecimalDataType;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
|
||||
|
@ -125,7 +124,7 @@ public abstract class AbstractKuduProcessor extends AbstractProcessor {
|
|||
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
|
||||
.build();
|
||||
|
||||
private static final FieldConverter<Object, Timestamp> TIMESTAMP_FIELD_CONVERTER = new ObjectTimestampFieldConverter();
|
||||
private static final FieldConverter<Object, Timestamp> TIMESTAMP_FIELD_CONVERTER = StandardFieldConverterRegistry.getRegistry().getFieldConverter(Timestamp.class);
|
||||
/** Timestamp Pattern overrides default RecordFieldType.TIMESTAMP pattern of yyyy-MM-dd HH:mm:ss with optional microseconds */
|
||||
private static final String MICROSECOND_TIMESTAMP_PATTERN = "yyyy-MM-dd HH:mm:ss[.SSSSSS]";
|
||||
|
||||
|
@ -328,7 +327,8 @@ public abstract class AbstractKuduProcessor extends AbstractProcessor {
|
|||
* @return Date object or null when value is null
|
||||
*/
|
||||
private Date getDate(final Object value, final String recordFieldName, final String format) {
|
||||
final LocalDate localDate = DataTypeUtils.toLocalDate(value, () -> DataTypeUtils.getDateTimeFormatter(format, ZoneId.systemDefault()), recordFieldName);
|
||||
final FieldConverter<Object, LocalDate> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(LocalDate.class);
|
||||
final LocalDate localDate = converter.convertField(value, Optional.ofNullable(format), recordFieldName);
|
||||
return Date.valueOf(localDate);
|
||||
}
|
||||
|
||||
|
|
|
@ -21,8 +21,9 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.math.BigDecimal;
|
||||
import java.sql.Timestamp;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.ZoneId;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -423,17 +424,17 @@ public class TestPutKudu {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testBuildPartialRowWithDateDefaultTimeZone() throws ParseException {
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat(ISO_8601_YEAR_MONTH_DAY_PATTERN);
|
||||
final java.util.Date dateFieldValue = dateFormat.parse(ISO_8601_YEAR_MONTH_DAY);
|
||||
public void testBuildPartialRowWithDateDefaultTimeZone() {
|
||||
final Instant localDate = LocalDate.parse(ISO_8601_YEAR_MONTH_DAY).atStartOfDay().atZone(ZoneId.systemDefault()).toInstant();
|
||||
final java.util.Date dateFieldValue = java.util.Date.from(localDate);
|
||||
|
||||
assertPartialRowDateFieldEquals(dateFieldValue);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildPartialRowWithDateToString() throws ParseException {
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat(ISO_8601_YEAR_MONTH_DAY_PATTERN);
|
||||
final java.util.Date dateFieldValue = dateFormat.parse(ISO_8601_YEAR_MONTH_DAY);
|
||||
public void testBuildPartialRowWithDateToString() {
|
||||
final Instant localDate = LocalDate.parse(ISO_8601_YEAR_MONTH_DAY).atStartOfDay().atZone(ZoneId.systemDefault()).toInstant();
|
||||
final java.util.Date dateFieldValue = java.util.Date.from(localDate);
|
||||
|
||||
final PartialRow row = buildPartialRowDateField(dateFieldValue, Type.STRING);
|
||||
final String column = row.getString(DATE_FIELD);
|
||||
|
|
|
@ -53,16 +53,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
public abstract class AbstractMongoProcessor extends AbstractProcessor {
|
||||
static final String WRITE_CONCERN_ACKNOWLEDGED = "ACKNOWLEDGED";
|
||||
static final String WRITE_CONCERN_UNACKNOWLEDGED = "UNACKNOWLEDGED";
|
||||
static final String WRITE_CONCERN_FSYNCED = "FSYNCED";
|
||||
static final String WRITE_CONCERN_JOURNALED = "JOURNALED";
|
||||
static final String WRITE_CONCERN_REPLICA_ACKNOWLEDGED = "REPLICA_ACKNOWLEDGED";
|
||||
static final String WRITE_CONCERN_MAJORITY = "MAJORITY";
|
||||
static final String WRITE_CONCERN_W1 = "W1";
|
||||
static final String WRITE_CONCERN_W2 = "W2";
|
||||
static final String WRITE_CONCERN_W3 = "W3";
|
||||
|
||||
protected static final String JSON_TYPE_EXTENDED = "Extended";
|
||||
protected static final String JSON_TYPE_STANDARD = "Standard";
|
||||
protected static final AllowableValue JSON_EXTENDED = new AllowableValue(JSON_TYPE_EXTENDED, "Extended JSON",
|
||||
|
@ -147,8 +137,7 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
|
|||
.name("mongo-date-format")
|
||||
.displayName("Date Format")
|
||||
.description("The date format string to use for formatting Date fields that are returned from Mongo. It is only " +
|
||||
"applied when the JSON output format is set to Standard JSON. Full documentation for format characters can be " +
|
||||
"found here: https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/text/SimpleDateFormat.html")
|
||||
"applied when the JSON output format is set to Standard JSON.")
|
||||
.defaultValue("yyyy-MM-dd'T'HH:mm:ss'Z'")
|
||||
.addValidator((subject, input, context) -> {
|
||||
ValidationResult.Builder result = new ValidationResult.Builder()
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.junit.jupiter.api.AfterEach;
|
|||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collection;
|
||||
|
@ -184,10 +183,8 @@ public class GetMongoIT extends AbstractMongoIT {
|
|||
byte[] raw = runner.getContentAsByteArray(flowFiles.get(0));
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
Map<String, Object> parsed = mapper.readValue(raw, Map.class);
|
||||
SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd");
|
||||
|
||||
assertTrue(parsed.get("date_field").getClass() == String.class);
|
||||
assertTrue(((String)parsed.get("date_field")).startsWith(format.format(CAL.getTime())));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -33,7 +33,9 @@ import org.junit.jupiter.api.BeforeEach;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Calendar;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
|
@ -53,7 +55,7 @@ public class RunMongoAggregationIT extends AbstractMongoIT {
|
|||
private TestRunner runner;
|
||||
private MongoClient mongoClient;
|
||||
private Map<String, Integer> mappings;
|
||||
private Calendar now = Calendar.getInstance();
|
||||
private final Calendar now = Calendar.getInstance();
|
||||
private MongoDBControllerService clientService;
|
||||
|
||||
@BeforeEach
|
||||
|
@ -184,11 +186,10 @@ public class RunMongoAggregationIT extends AbstractMongoIT {
|
|||
|
||||
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(RunMongoAggregation.REL_RESULTS);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'");
|
||||
for (MockFlowFile mockFlowFile : flowFiles) {
|
||||
byte[] raw = runner.getContentAsByteArray(mockFlowFile);
|
||||
Map<String, List<String>> read = mapper.readValue(raw, Map.class);
|
||||
assertTrue(read.get("myArray").get(1).equalsIgnoreCase( format.format(now.getTime())));
|
||||
assertNotNull(read.get("myArray").get(1));
|
||||
}
|
||||
|
||||
runner.clearTransferState();
|
||||
|
@ -225,15 +226,15 @@ public class RunMongoAggregationIT extends AbstractMongoIT {
|
|||
@Test
|
||||
public void testExtendedJsonSupport() throws Exception {
|
||||
String pattern = "yyyy-MM-dd'T'HH:mm:ss'Z'";
|
||||
SimpleDateFormat simpleDateFormat = new SimpleDateFormat(pattern);
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(pattern);
|
||||
//Let's put this a week from now to make sure that we're not getting too close to
|
||||
//the creation date
|
||||
Date nowish = new Date(now.getTime().getTime() + (7 * 24 * 60 * 60 * 1000));
|
||||
OffsetDateTime nowish = new Date(now.getTime().getTime() + (7 * 24 * 60 * 60 * 1000)).toInstant().atOffset(ZoneOffset.UTC);
|
||||
|
||||
final String queryInput = "[\n" +
|
||||
" {\n" +
|
||||
" \"$match\": {\n" +
|
||||
" \"date\": { \"$gte\": { \"$date\": \"2019-01-01T00:00:00Z\" }, \"$lte\": { \"$date\": \"" + simpleDateFormat.format(nowish) + "\" } }\n" +
|
||||
" \"date\": { \"$gte\": { \"$date\": \"2019-01-01T00:00:00Z\" }, \"$lte\": { \"$date\": \"" + dateTimeFormatter.format(nowish) + "\" } }\n" +
|
||||
" }\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
|
|
|
@ -31,11 +31,10 @@ import org.apache.poi.ss.usermodel.Row;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.apache.commons.lang3.StringUtils.isEmpty;
|
||||
|
@ -43,9 +42,6 @@ import static org.apache.commons.lang3.StringUtils.isEmpty;
|
|||
public class ExcelRecordReader implements RecordReader {
|
||||
private final RowIterator rowIterator;
|
||||
private final RecordSchema schema;
|
||||
private final Supplier<DateFormat> LAZY_DATE_FORMAT;
|
||||
private final Supplier<DateFormat> LAZY_TIME_FORMAT;
|
||||
private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
|
||||
private final String dateFormat;
|
||||
private final String timeFormat;
|
||||
private final String timestampFormat;
|
||||
|
@ -55,26 +51,20 @@ public class ExcelRecordReader implements RecordReader {
|
|||
|
||||
if (isEmpty(configuration.getDateFormat())) {
|
||||
this.dateFormat = null;
|
||||
LAZY_DATE_FORMAT = null;
|
||||
} else {
|
||||
this.dateFormat = configuration.getDateFormat();
|
||||
LAZY_DATE_FORMAT = () -> DataTypeUtils.getDateFormat(dateFormat);
|
||||
}
|
||||
|
||||
if (isEmpty(configuration.getTimeFormat())) {
|
||||
this.timeFormat = null;
|
||||
LAZY_TIME_FORMAT = null;
|
||||
} else {
|
||||
this.timeFormat = configuration.getTimeFormat();
|
||||
LAZY_TIME_FORMAT = () -> DataTypeUtils.getDateFormat(timeFormat);
|
||||
}
|
||||
|
||||
if (isEmpty(configuration.getTimestampFormat())) {
|
||||
this.timestampFormat = null;
|
||||
LAZY_TIMESTAMP_FORMAT = null;
|
||||
} else {
|
||||
this.timestampFormat = configuration.getTimestampFormat();
|
||||
LAZY_TIMESTAMP_FORMAT = () -> DataTypeUtils.getDateFormat(timestampFormat);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -139,18 +129,11 @@ public class ExcelRecordReader implements RecordReader {
|
|||
|
||||
private static Object getCellValue(Cell cell) {
|
||||
if (cell != null) {
|
||||
switch (cell.getCellType()) {
|
||||
case _NONE:
|
||||
case BLANK:
|
||||
case ERROR:
|
||||
case FORMULA:
|
||||
case STRING:
|
||||
return cell.getStringCellValue();
|
||||
case NUMERIC:
|
||||
return DateUtil.isCellDateFormatted(cell) ? cell.getDateCellValue() : cell.getNumericCellValue();
|
||||
case BOOLEAN:
|
||||
return cell.getBooleanCellValue();
|
||||
}
|
||||
return switch (cell.getCellType()) {
|
||||
case _NONE, BLANK, ERROR, FORMULA, STRING -> cell.getStringCellValue();
|
||||
case NUMERIC -> DateUtil.isCellDateFormatted(cell) ? cell.getDateCellValue() : cell.getNumericCellValue();
|
||||
case BOOLEAN -> cell.getBooleanCellValue();
|
||||
};
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -160,7 +143,7 @@ public class ExcelRecordReader implements RecordReader {
|
|||
return value;
|
||||
}
|
||||
|
||||
return DataTypeUtils.convertType(value, dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName);
|
||||
}
|
||||
|
||||
private Object convertSimpleIfPossible(final Object value, final DataType dataType, final String fieldName) {
|
||||
|
@ -181,22 +164,22 @@ public class ExcelRecordReader implements RecordReader {
|
|||
case CHAR:
|
||||
case SHORT:
|
||||
if (DataTypeUtils.isCompatibleDataType(value, dataType)) {
|
||||
return DataTypeUtils.convertType(value, dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName);
|
||||
}
|
||||
break;
|
||||
case DATE:
|
||||
if (DataTypeUtils.isDateTypeCompatible(value, dateFormat)) {
|
||||
return DataTypeUtils.convertType(value, dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName);
|
||||
}
|
||||
break;
|
||||
case TIME:
|
||||
if (DataTypeUtils.isTimeTypeCompatible(value, timeFormat)) {
|
||||
return DataTypeUtils.convertType(value, dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName);
|
||||
}
|
||||
break;
|
||||
case TIMESTAMP:
|
||||
if (DataTypeUtils.isTimestampTypeCompatible(value, timestampFormat)) {
|
||||
return DataTypeUtils.convertType(value, dataType, LAZY_DATE_FORMAT, LAZY_TIME_FORMAT, LAZY_TIMESTAMP_FORMAT, fieldName);
|
||||
return DataTypeUtils.convertType(value, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -63,7 +63,8 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.DateFormat;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -81,6 +82,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
|
|||
protected static final String LAST_EVENT_ID_KEY = "last_event_id";
|
||||
protected static final String DESTINATION_URL_PATH = "/nifi";
|
||||
protected static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
|
||||
protected static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(TIMESTAMP_FORMAT).withZone(ZoneOffset.UTC);
|
||||
|
||||
static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
|
||||
.name("record-writer")
|
||||
|
@ -122,7 +124,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
|
|||
return properties;
|
||||
}
|
||||
|
||||
public void setup(final PropertyContext reportContext) throws IOException {
|
||||
public void setup(final PropertyContext reportContext) {
|
||||
if (siteToSiteClient == null) {
|
||||
siteToSiteClient = SiteToSiteUtils.getClient(reportContext, getLogger(), null);
|
||||
}
|
||||
|
@ -217,10 +219,6 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
|
|||
private final JsonNode firstJsonNode;
|
||||
private boolean firstObjectConsumed = false;
|
||||
|
||||
private final Supplier<DateFormat> dateFormat = () -> DataTypeUtils.getDateFormat(RecordFieldType.DATE.getDefaultFormat());
|
||||
private final Supplier<DateFormat> timeFormat = () -> DataTypeUtils.getDateFormat(RecordFieldType.TIME.getDefaultFormat());
|
||||
private final Supplier<DateFormat> timestampFormat = () -> DataTypeUtils.getDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat());
|
||||
|
||||
public JsonRecordReader(final InputStream in, RecordSchema recordSchema) throws IOException, MalformedRecordException {
|
||||
this.recordSchema = recordSchema;
|
||||
try {
|
||||
|
@ -381,8 +379,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
|
|||
case TIME:
|
||||
case TIMESTAMP: {
|
||||
final Object rawValue = getRawNodeValue(fieldNode, null);
|
||||
final Object converted = DataTypeUtils.convertType(rawValue, desiredType, dateFormat, timeFormat, timestampFormat, fieldName);
|
||||
return converted;
|
||||
return DataTypeUtils.convertType(rawValue, desiredType, fieldName);
|
||||
}
|
||||
case MAP: {
|
||||
final DataType valueType = ((MapDataType) desiredType).getValueType();
|
||||
|
|
|
@ -40,15 +40,12 @@ import javax.json.JsonObject;
|
|||
import javax.json.JsonObjectBuilder;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -121,16 +118,13 @@ public class SiteToSiteBulletinReportingTask extends AbstractSiteToSiteReporting
|
|||
final JsonBuilderFactory factory = Json.createBuilderFactory(config);
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
|
||||
final DateFormat df = new SimpleDateFormat(TIMESTAMP_FORMAT);
|
||||
df.setTimeZone(TimeZone.getTimeZone("Z"));
|
||||
|
||||
final long start = System.nanoTime();
|
||||
|
||||
// Create a JSON array of all the events in the current batch
|
||||
final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
|
||||
for (final Bulletin bulletin : bulletins) {
|
||||
if (bulletin.getId() > lastSentBulletinId) {
|
||||
arrayBuilder.add(serialize(builder, bulletin, df, platform, nodeId, allowNullValues));
|
||||
arrayBuilder.add(serialize(builder, bulletin, platform, nodeId, allowNullValues));
|
||||
}
|
||||
}
|
||||
final JsonArray jsonArray = arrayBuilder.build();
|
||||
|
@ -176,7 +170,7 @@ public class SiteToSiteBulletinReportingTask extends AbstractSiteToSiteReporting
|
|||
lastSentBulletinId = currMaxId;
|
||||
}
|
||||
|
||||
private JsonObject serialize(final JsonObjectBuilder builder, final Bulletin bulletin, final DateFormat df,
|
||||
private JsonObject serialize(final JsonObjectBuilder builder, final Bulletin bulletin,
|
||||
final String platform, final String nodeIdentifier, Boolean allowNullValues) {
|
||||
|
||||
addField(builder, "objectId", UUID.randomUUID().toString(), allowNullValues);
|
||||
|
@ -193,7 +187,7 @@ public class SiteToSiteBulletinReportingTask extends AbstractSiteToSiteReporting
|
|||
addField(builder, "bulletinSourceId", bulletin.getSourceId(), allowNullValues);
|
||||
addField(builder, "bulletinSourceName", bulletin.getSourceName(), allowNullValues);
|
||||
addField(builder, "bulletinSourceType", bulletin.getSourceType() == null ? null : bulletin.getSourceType().name(), allowNullValues);
|
||||
addField(builder, "bulletinTimestamp", df.format(bulletin.getTimestamp()), allowNullValues);
|
||||
addField(builder, "bulletinTimestamp", DATE_TIME_FORMATTER.format(bulletin.getTimestamp().toInstant()), allowNullValues);
|
||||
addField(builder, "bulletinFlowFileUuid", bulletin.getFlowFileUuid(), allowNullValues);
|
||||
|
||||
return builder.build();
|
||||
|
|
|
@ -22,8 +22,7 @@ import java.io.InputStream;
|
|||
import java.net.MalformedURLException;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -31,7 +30,6 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -189,7 +187,7 @@ public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReporti
|
|||
}
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(final ConfigurationContext context) throws IOException {
|
||||
public void onScheduled(final ConfigurationContext context) {
|
||||
consumer = new ProvenanceEventConsumer();
|
||||
consumer.setStartPositionValue(context.getProperty(START_POSITION).getValue());
|
||||
consumer.setBatchSize(context.getProperty(SiteToSiteUtils.BATCH_SIZE).asInteger());
|
||||
|
@ -289,9 +287,6 @@ public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReporti
|
|||
final JsonBuilderFactory factory = Json.createBuilderFactory(config);
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
|
||||
final DateFormat df = new SimpleDateFormat(TIMESTAMP_FORMAT);
|
||||
df.setTimeZone(TimeZone.getTimeZone("Z"));
|
||||
|
||||
consumer.consumeEvents(context, (mapHolder, events) -> {
|
||||
final long start = System.nanoTime();
|
||||
// Create a JSON array of all the events in the current batch
|
||||
|
@ -300,7 +295,7 @@ public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReporti
|
|||
final String componentName = mapHolder.getComponentName(event.getComponentId());
|
||||
final String processGroupId = mapHolder.getProcessGroupId(event.getComponentId(), event.getComponentType());
|
||||
final String processGroupName = mapHolder.getComponentName(processGroupId);
|
||||
arrayBuilder.add(serialize(factory, builder, event, df, componentName, processGroupId, processGroupName, hostname, url, rootGroupName, platform, nodeId, allowNullValues));
|
||||
arrayBuilder.add(serialize(factory, builder, event, componentName, processGroupId, processGroupName, hostname, url, rootGroupName, platform, nodeId, allowNullValues));
|
||||
}
|
||||
final JsonArray jsonArray = arrayBuilder.build();
|
||||
|
||||
|
@ -346,14 +341,14 @@ public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReporti
|
|||
}
|
||||
|
||||
|
||||
private JsonObject serialize(final JsonBuilderFactory factory, final JsonObjectBuilder builder, final ProvenanceEventRecord event, final DateFormat df,
|
||||
private JsonObject serialize(final JsonBuilderFactory factory, final JsonObjectBuilder builder, final ProvenanceEventRecord event,
|
||||
final String componentName, final String processGroupId, final String processGroupName, final String hostname, final URL nifiUrl, final String applicationName,
|
||||
final String platform, final String nodeIdentifier, Boolean allowNullValues) {
|
||||
addField(builder, "eventId", UUID.randomUUID().toString(), allowNullValues);
|
||||
addField(builder, "eventOrdinal", event.getEventId(), allowNullValues);
|
||||
addField(builder, "eventType", event.getEventType().name(), allowNullValues);
|
||||
addField(builder, "timestampMillis", event.getEventTime(), allowNullValues);
|
||||
addField(builder, "timestamp", df.format(event.getEventTime()), allowNullValues);
|
||||
addField(builder, "timestamp", DATE_TIME_FORMATTER.format(Instant.ofEpochMilli(event.getEventTime())), allowNullValues);
|
||||
addField(builder, "durationMillis", event.getEventDuration(), allowNullValues);
|
||||
addField(builder, "lineageStart", event.getLineageStartDate(), allowNullValues);
|
||||
addField(builder, "details", event.getDetails(), allowNullValues);
|
||||
|
|
|
@ -22,15 +22,12 @@ import java.io.InputStream;
|
|||
import java.net.MalformedURLException;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TimeZone;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -146,11 +143,8 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
final Map<String, ?> config = Collections.emptyMap();
|
||||
final JsonBuilderFactory factory = Json.createBuilderFactory(config);
|
||||
|
||||
final DateFormat df = new SimpleDateFormat(TIMESTAMP_FORMAT);
|
||||
df.setTimeZone(TimeZone.getTimeZone("Z"));
|
||||
|
||||
final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
|
||||
serializeProcessGroupStatus(arrayBuilder, factory, procGroupStatus, df,
|
||||
serializeProcessGroupStatus(arrayBuilder, factory, procGroupStatus,
|
||||
hostname, rootGroupName, platform, null, new Date(), allowNullValues);
|
||||
|
||||
final JsonArray jsonArray = arrayBuilder.build();
|
||||
|
@ -235,8 +229,6 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
* The JSON Builder Factory
|
||||
* @param status
|
||||
* The ProcessGroupStatus
|
||||
* @param df
|
||||
* A date format
|
||||
* @param hostname
|
||||
* The current hostname
|
||||
* @param applicationName
|
||||
|
@ -251,7 +243,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
* Allow null values
|
||||
*/
|
||||
private void serializeProcessGroupStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory,
|
||||
final ProcessGroupStatus status, final DateFormat df, final String hostname, final String applicationName,
|
||||
final ProcessGroupStatus status, final String hostname, final String applicationName,
|
||||
final String platform, final ProcessGroupStatus parent, final Date currentDate, Boolean allowNullValues) {
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
final String componentType = parent == null ? "RootProcessGroup" : "ProcessGroup";
|
||||
|
@ -262,7 +254,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
}
|
||||
|
||||
if (componentMatchesFilters(componentType, componentName)) {
|
||||
addCommonFields(builder, df, hostname, applicationName, platform, parent, currentDate,
|
||||
addCommonFields(builder, hostname, applicationName, platform, parent, currentDate,
|
||||
componentType, componentName, allowNullValues);
|
||||
|
||||
addField(builder, "componentId", status.getId(), allowNullValues);
|
||||
|
@ -291,40 +283,40 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
|
||||
processGroupIDToPath.put(childGroupStatus.getId(), processGroupIDToPath.get(status.getId()) + " / " + childGroupStatus.getName());
|
||||
|
||||
serializeProcessGroupStatus(arrayBuilder, factory, childGroupStatus, df, hostname,
|
||||
serializeProcessGroupStatus(arrayBuilder, factory, childGroupStatus, hostname,
|
||||
applicationName, platform, status, currentDate, allowNullValues);
|
||||
}
|
||||
for(ProcessorStatus processorStatus : status.getProcessorStatus()) {
|
||||
serializeProcessorStatus(arrayBuilder, factory, processorStatus, df, hostname,
|
||||
serializeProcessorStatus(arrayBuilder, factory, processorStatus, hostname,
|
||||
applicationName, platform, status, currentDate, allowNullValues);
|
||||
}
|
||||
for(ConnectionStatus connectionStatus : status.getConnectionStatus()) {
|
||||
serializeConnectionStatus(arrayBuilder, factory, connectionStatus, df, hostname,
|
||||
serializeConnectionStatus(arrayBuilder, factory, connectionStatus, hostname,
|
||||
applicationName, platform, status, currentDate, allowNullValues);
|
||||
}
|
||||
for(PortStatus portStatus : status.getInputPortStatus()) {
|
||||
serializePortStatus("InputPort", arrayBuilder, factory, portStatus, df,
|
||||
serializePortStatus("InputPort", arrayBuilder, factory, portStatus,
|
||||
hostname, applicationName, platform, status, currentDate, allowNullValues);
|
||||
}
|
||||
for(PortStatus portStatus : status.getOutputPortStatus()) {
|
||||
serializePortStatus("OutputPort", arrayBuilder, factory, portStatus, df,
|
||||
serializePortStatus("OutputPort", arrayBuilder, factory, portStatus,
|
||||
hostname, applicationName, platform, status, currentDate, allowNullValues);
|
||||
}
|
||||
for(RemoteProcessGroupStatus remoteProcessGroupStatus : status.getRemoteProcessGroupStatus()) {
|
||||
serializeRemoteProcessGroupStatus(arrayBuilder, factory, remoteProcessGroupStatus, df, hostname,
|
||||
serializeRemoteProcessGroupStatus(arrayBuilder, factory, remoteProcessGroupStatus, hostname,
|
||||
applicationName, platform, status, currentDate, allowNullValues);
|
||||
}
|
||||
}
|
||||
|
||||
private void serializeRemoteProcessGroupStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory,
|
||||
final RemoteProcessGroupStatus status, final DateFormat df, final String hostname, final String applicationName,
|
||||
final RemoteProcessGroupStatus status, final String hostname, final String applicationName,
|
||||
final String platform, final ProcessGroupStatus parent, final Date currentDate, final Boolean allowNullValues) {
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
final String componentType = "RemoteProcessGroup";
|
||||
final String componentName = status.getName();
|
||||
|
||||
if (componentMatchesFilters(componentType, componentName)) {
|
||||
addCommonFields(builder, df, hostname, applicationName, platform, parent, currentDate,
|
||||
addCommonFields(builder, hostname, applicationName, platform, parent, currentDate,
|
||||
componentType, componentName, allowNullValues);
|
||||
|
||||
addField(builder, "componentId", status.getId(), allowNullValues);
|
||||
|
@ -344,12 +336,12 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
}
|
||||
|
||||
private void serializePortStatus(final String componentType, final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory, final PortStatus status,
|
||||
final DateFormat df, final String hostname, final String applicationName, final String platform, final ProcessGroupStatus parent, final Date currentDate, final Boolean allowNullValues) {
|
||||
final String hostname, final String applicationName, final String platform, final ProcessGroupStatus parent, final Date currentDate, final Boolean allowNullValues) {
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
final String componentName = status.getName();
|
||||
|
||||
if (componentMatchesFilters(componentType, componentName)) {
|
||||
addCommonFields(builder, df, hostname, applicationName, platform, parent, currentDate,
|
||||
addCommonFields(builder, hostname, applicationName, platform, parent, currentDate,
|
||||
componentType, componentName, allowNullValues);
|
||||
|
||||
addField(builder, "componentId", status.getId(), allowNullValues);
|
||||
|
@ -369,14 +361,14 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
}
|
||||
}
|
||||
|
||||
private void serializeConnectionStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory, final ConnectionStatus status, final DateFormat df,
|
||||
private void serializeConnectionStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory, final ConnectionStatus status,
|
||||
final String hostname, final String applicationName, final String platform, final ProcessGroupStatus parent, final Date currentDate, final Boolean allowNullValues) {
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
final String componentType = "Connection";
|
||||
final String componentName = status.getName();
|
||||
|
||||
if (componentMatchesFilters(componentType, componentName)) {
|
||||
addCommonFields(builder, df, hostname, applicationName, platform, parent, currentDate,
|
||||
addCommonFields(builder, hostname, applicationName, platform, parent, currentDate,
|
||||
componentType, componentName, allowNullValues);
|
||||
|
||||
addField(builder, "componentId", status.getId(), allowNullValues);
|
||||
|
@ -402,14 +394,14 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
}
|
||||
}
|
||||
|
||||
private void serializeProcessorStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory, final ProcessorStatus status, final DateFormat df,
|
||||
private void serializeProcessorStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory, final ProcessorStatus status,
|
||||
final String hostname, final String applicationName, final String platform, final ProcessGroupStatus parent, final Date currentDate, final Boolean allowNullValues) {
|
||||
final JsonObjectBuilder builder = factory.createObjectBuilder();
|
||||
final String componentType = "Processor";
|
||||
final String componentName = status.getName();
|
||||
|
||||
if (componentMatchesFilters(componentType, componentName)) {
|
||||
addCommonFields(builder, df, hostname, applicationName, platform, parent, currentDate, componentType, componentName, allowNullValues);
|
||||
addCommonFields(builder, hostname, applicationName, platform, parent, currentDate, componentType, componentName, allowNullValues);
|
||||
|
||||
addField(builder, "componentId", status.getId(), allowNullValues);
|
||||
addField(builder, "processorType", status.getType(), allowNullValues);
|
||||
|
@ -437,12 +429,12 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa
|
|||
}
|
||||
}
|
||||
|
||||
private void addCommonFields(final JsonObjectBuilder builder, final DateFormat df, final String hostname,
|
||||
private void addCommonFields(final JsonObjectBuilder builder, final String hostname,
|
||||
final String applicationName, final String platform, final ProcessGroupStatus parent, final Date currentDate,
|
||||
final String componentType, final String componentName, Boolean allowNullValues) {
|
||||
addField(builder, "statusId", UUID.randomUUID().toString(), allowNullValues);
|
||||
addField(builder, "timestampMillis", currentDate.getTime(), allowNullValues);
|
||||
addField(builder, "timestamp", df.format(currentDate), allowNullValues);
|
||||
addField(builder, "timestamp", DATE_TIME_FORMATTER.format(currentDate.toInstant()), allowNullValues);
|
||||
addField(builder, "actorHostname", hostname, allowNullValues);
|
||||
addField(builder, "componentType", componentType, allowNullValues);
|
||||
addField(builder, "componentName", componentName, allowNullValues);
|
||||
|
|
|
@ -58,8 +58,8 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -213,7 +213,7 @@ public class GetSmbFile extends AbstractProcessor {
|
|||
public static final String FILE_SIZE_ATTRIBUTE = "file.size";
|
||||
|
||||
public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
|
||||
final static DateFormat dateFormatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
private static final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("All files are routed to success").build();
|
||||
|
||||
|
@ -486,9 +486,9 @@ public class GetSmbFile extends AbstractProcessor {
|
|||
attributes.put(CoreAttributes.FILENAME.key(), filename);
|
||||
attributes.put(CoreAttributes.PATH.key(), filePath);
|
||||
attributes.put(CoreAttributes.ABSOLUTE_PATH.key(), "\\\\" + hostname + "\\" + shareName + "\\" + file);
|
||||
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, dateFormatter.format(fileBasicInfo.getCreationTime().toDate()));
|
||||
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, dateFormatter.format(fileBasicInfo.getLastAccessTime().toDate()));
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, dateFormatter.format(fileBasicInfo.getLastWriteTime().toDate()));
|
||||
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, dateFormatter.format(fileBasicInfo.getCreationTime().toInstant().atZone(ZoneId.systemDefault())));
|
||||
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, dateFormatter.format(fileBasicInfo.getLastAccessTime().toInstant().atZone(ZoneId.systemDefault())));
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, dateFormatter.format(fileBasicInfo.getLastWriteTime().toInstant().atZone(ZoneId.systemDefault())));
|
||||
attributes.put(FILE_SIZE_ATTRIBUTE, String.valueOf(fileSize));
|
||||
attributes.put(HOSTNAME.getName(), hostname);
|
||||
attributes.put(SHARE.getName(), shareName);
|
||||
|
@ -497,12 +497,10 @@ public class GetSmbFile extends AbstractProcessor {
|
|||
session.getProvenanceReporter().receive(flowFile, uri.toString(), importMillis);
|
||||
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
logger.info("added {} to flow", new Object[]{flowFile});
|
||||
|
||||
} catch (SMBApiException e) {
|
||||
// do not fail whole batch if a single file cannot be accessed
|
||||
if (e.getStatus() == NtStatus.STATUS_SHARING_VIOLATION) {
|
||||
logger.info("Could not acquire sharing access for file {}", new Object[]{file});
|
||||
logger.info("Could not acquire sharing access for file {}", file);
|
||||
if (flowFile != null) {
|
||||
session.remove(flowFile);
|
||||
}
|
||||
|
@ -517,7 +515,7 @@ public class GetSmbFile extends AbstractProcessor {
|
|||
share.rm(file);
|
||||
}
|
||||
} catch (SMBApiException e) {
|
||||
logger.error("Could not remove file {}", new Object[]{file});
|
||||
logger.error("Could not remove file {}", file);
|
||||
}
|
||||
|
||||
if (!isScheduled()) { // if processor stopped, put the rest of the files back on the queue.
|
||||
|
|
|
@ -35,7 +35,6 @@ import java.io.InputStream;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
|
@ -169,7 +168,7 @@ public class GetSmbFileTest {
|
|||
@Test
|
||||
public void testOpenFileCalled() {
|
||||
FileIdBothDirectoryInformation file1 = mockFile(DIRECTORY, "file1.txt", "abc");
|
||||
mockDir(DIRECTORY, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
mockDir(DIRECTORY, new ArrayList<>(){{
|
||||
add(file1);
|
||||
}});
|
||||
testRunner.run();
|
||||
|
@ -182,7 +181,7 @@ public class GetSmbFileTest {
|
|||
testRunner.setProperty(GetSmbFile.IGNORE_HIDDEN_FILES, "true");
|
||||
FileIdBothDirectoryInformation file1 = mockFile(DIRECTORY, "file1.txt", "abc", FileAttributes.FILE_ATTRIBUTE_HIDDEN.getValue());
|
||||
FileIdBothDirectoryInformation file2 = mockFile(DIRECTORY, "file2.txt", "abc", FileAttributes.FILE_ATTRIBUTE_NORMAL.getValue());
|
||||
mockDir(DIRECTORY, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
mockDir(DIRECTORY, new ArrayList<>(){{
|
||||
add(file1);
|
||||
add(file2);
|
||||
}});
|
||||
|
@ -194,7 +193,7 @@ public class GetSmbFileTest {
|
|||
@Test
|
||||
public void testFileFilter() {
|
||||
testRunner.setProperty(GetSmbFile.FILE_FILTER, "file[0-9]\\.txt");
|
||||
mockDir(DIRECTORY, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
mockDir(DIRECTORY, new ArrayList<>(){{
|
||||
add(mockFile(DIRECTORY, "something_else.txt", "abc"));
|
||||
add(mockFile(DIRECTORY, "file1.txt", "abc"));
|
||||
add(mockFile(DIRECTORY, "file2.txt", "abc"));
|
||||
|
@ -210,10 +209,10 @@ public class GetSmbFileTest {
|
|||
public void testNonRecurse() {
|
||||
testRunner.setProperty(GetSmbFile.RECURSE, "false");
|
||||
String subdir = DIRECTORY + "\\subdir1";
|
||||
mockDir(DIRECTORY, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
mockDir(DIRECTORY, new ArrayList<>(){{
|
||||
add(mockFile(DIRECTORY, "file1.txt", "abc"));
|
||||
add(mockFile(DIRECTORY, "file2.txt", "abc"));
|
||||
add(mockDir(subdir, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
add(mockDir(subdir, new ArrayList<>(){{
|
||||
add(mockFile(subdir, "file3.txt", "abc"));
|
||||
}}));
|
||||
}});
|
||||
|
@ -229,10 +228,10 @@ public class GetSmbFileTest {
|
|||
public void testRecurse() {
|
||||
testRunner.setProperty(GetSmbFile.RECURSE, "true");
|
||||
String subdir = DIRECTORY + "\\subdir1";
|
||||
mockDir(DIRECTORY, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
mockDir(DIRECTORY, new ArrayList<>(){{
|
||||
add(mockFile(DIRECTORY, "file1.txt", "abc"));
|
||||
add(mockFile(DIRECTORY, "file2.txt", "abc"));
|
||||
add(mockDir(subdir, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
add(mockDir(subdir, new ArrayList<>(){{
|
||||
add(mockFile(subdir, "file3.txt", "abc"));
|
||||
}}));
|
||||
}});
|
||||
|
@ -252,14 +251,14 @@ public class GetSmbFileTest {
|
|||
String subdir1 = DIRECTORY + "\\subdir1";
|
||||
String subdir2 = DIRECTORY + "\\subdir2";
|
||||
String subdir3 = DIRECTORY + "\\foo";
|
||||
mockDir(DIRECTORY, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
add(mockDir(subdir1, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
mockDir(DIRECTORY, new ArrayList<>(){{
|
||||
add(mockDir(subdir1, new ArrayList<>(){{
|
||||
add(mockFile(subdir1, "file1.txt", "abc"));
|
||||
}}));
|
||||
add(mockDir(subdir2, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
add(mockDir(subdir2, new ArrayList<>(){{
|
||||
add(mockFile(subdir2, "file2.txt", "abc"));
|
||||
}}));
|
||||
add(mockDir(subdir3, new ArrayList<FileIdBothDirectoryInformation>(){{
|
||||
add(mockDir(subdir3, new ArrayList<>(){{
|
||||
add(mockFile(subdir3, "file3.txt", "abc"));
|
||||
}}));
|
||||
}});
|
||||
|
@ -295,7 +294,7 @@ public class GetSmbFileTest {
|
|||
final List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(GetSmbFile.REL_SUCCESS);
|
||||
final List<String> flowFileNames = flowFiles.stream()
|
||||
.map(flowFile -> flowFile.getAttribute(CoreAttributes.FILENAME.key()))
|
||||
.collect(Collectors.toList());
|
||||
.toList();
|
||||
|
||||
for (int i = 0; i < totalSize; i++) {
|
||||
final String flowFileName = flowFileNames.get(0);
|
||||
|
|
|
@ -20,18 +20,19 @@ package org.apache.nifi.processors.solr;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
|
@ -151,15 +152,12 @@ public class GetSolr extends SolrProcessor {
|
|||
.description("The results of querying Solr")
|
||||
.build();
|
||||
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.US);
|
||||
|
||||
private final AtomicBoolean clearState = new AtomicBoolean(false);
|
||||
private final AtomicBoolean dateFieldNotInSpecifiedFieldsList = new AtomicBoolean(false);
|
||||
private volatile String id_field = null;
|
||||
|
||||
private static final SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.US);
|
||||
static {
|
||||
df.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
}
|
||||
|
||||
private Set<Relationship> relationships;
|
||||
private List<PropertyDescriptor> descriptors;
|
||||
|
||||
|
@ -167,29 +165,27 @@ public class GetSolr extends SolrProcessor {
|
|||
protected void init(final ProcessorInitializationContext context) {
|
||||
super.init(context);
|
||||
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(SOLR_TYPE);
|
||||
descriptors.add(SOLR_LOCATION);
|
||||
descriptors.add(COLLECTION);
|
||||
descriptors.add(RETURN_TYPE);
|
||||
descriptors.add(RECORD_WRITER);
|
||||
descriptors.add(SOLR_QUERY);
|
||||
descriptors.add(DATE_FIELD);
|
||||
descriptors.add(DATE_FILTER);
|
||||
descriptors.add(RETURN_FIELDS);
|
||||
descriptors.add(BATCH_SIZE);
|
||||
descriptors.add(KERBEROS_USER_SERVICE);
|
||||
descriptors.add(BASIC_USERNAME);
|
||||
descriptors.add(BASIC_PASSWORD);
|
||||
descriptors.add(SSL_CONTEXT_SERVICE);
|
||||
descriptors.add(SOLR_SOCKET_TIMEOUT);
|
||||
descriptors.add(SOLR_CONNECTION_TIMEOUT);
|
||||
descriptors.add(SOLR_MAX_CONNECTIONS_PER_HOST);
|
||||
this.descriptors = Collections.unmodifiableList(descriptors);
|
||||
this.descriptors = List.of(
|
||||
SOLR_TYPE,
|
||||
SOLR_LOCATION,
|
||||
COLLECTION,
|
||||
RETURN_TYPE,
|
||||
RECORD_WRITER,
|
||||
SOLR_QUERY,
|
||||
DATE_FIELD,
|
||||
DATE_FILTER,
|
||||
RETURN_FIELDS,
|
||||
BATCH_SIZE,
|
||||
KERBEROS_USER_SERVICE,
|
||||
BASIC_USERNAME,
|
||||
BASIC_PASSWORD,
|
||||
SSL_CONTEXT_SERVICE,
|
||||
SOLR_SOCKET_TIMEOUT,
|
||||
SOLR_CONNECTION_TIMEOUT,
|
||||
SOLR_MAX_CONNECTIONS_PER_HOST
|
||||
);
|
||||
|
||||
final Set<Relationship> relationships = new HashSet<>();
|
||||
relationships.add(REL_SUCCESS);
|
||||
this.relationships = Collections.unmodifiableSet(relationships);
|
||||
this.relationships = Set.of(REL_SUCCESS);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -202,7 +198,7 @@ public class GetSolr extends SolrProcessor {
|
|||
return this.descriptors;
|
||||
}
|
||||
|
||||
final static Set<String> propertyNamesForActivatingClearState = new HashSet<String>();
|
||||
final static Set<String> propertyNamesForActivatingClearState = new HashSet<>();
|
||||
static {
|
||||
propertyNamesForActivatingClearState.add(SOLR_TYPE.getName());
|
||||
propertyNamesForActivatingClearState.add(SOLR_LOCATION.getName());
|
||||
|
@ -336,7 +332,8 @@ public class GetSolr extends SolrProcessor {
|
|||
|
||||
if (response.getResults().size() > 0) {
|
||||
final SolrDocument lastSolrDocument = documentList.get(response.getResults().size()-1);
|
||||
final String latestDateValue = df.format(lastSolrDocument.get(dateField));
|
||||
final Object dateObject = lastSolrDocument.get(dateField);
|
||||
final String latestDateValue = getDateFormatted(dateObject);
|
||||
final String newCursorMark = response.getNextCursorMark();
|
||||
|
||||
solrQuery.setParam(CursorMarkParams.CURSOR_MARK_PARAM, newCursorMark);
|
||||
|
@ -407,6 +404,18 @@ public class GetSolr extends SolrProcessor {
|
|||
}
|
||||
}
|
||||
|
||||
private String getDateFormatted(final Object dateObject) {
|
||||
final String formatted;
|
||||
|
||||
if (dateObject instanceof Date date) {
|
||||
final OffsetDateTime dateTime = date.toInstant().atOffset(ZoneOffset.UTC);
|
||||
formatted = DATE_TIME_FORMATTER.format(dateTime);
|
||||
} else if (dateObject == null) {
|
||||
formatted = null;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Date Object Type [%s] not supported".formatted(dateObject.getClass()));
|
||||
}
|
||||
|
||||
return formatted;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,6 +57,8 @@ 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.serialization.record.RecordSet;
|
||||
import org.apache.nifi.serialization.record.field.FieldConverter;
|
||||
import org.apache.nifi.serialization.record.field.StandardFieldConverterRegistry;
|
||||
import org.apache.nifi.serialization.record.type.ChoiceDataType;
|
||||
import org.apache.nifi.serialization.record.util.DataTypeUtils;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
|
@ -350,7 +352,8 @@ public class SolrUtils {
|
|||
|
||||
switch (chosenDataType.getFieldType()) {
|
||||
case DATE: {
|
||||
final String stringValue = DataTypeUtils.toString(coercedValue, () -> DataTypeUtils.getDateFormat(RecordFieldType.DATE.getDefaultFormat()));
|
||||
final FieldConverter<Object, String> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class);
|
||||
final String stringValue = converter.convertField(coercedValue, Optional.ofNullable(RecordFieldType.DATE.getDefaultFormat()), fieldName);
|
||||
if (DataTypeUtils.isLongTypeCompatible(stringValue)) {
|
||||
LocalDate localDate = getLocalDateFromEpochTime(fieldName, coercedValue);
|
||||
addFieldToSolrDocument(inputDocument,fieldName,localDate.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)+'Z',fieldsToIndex);
|
||||
|
@ -360,7 +363,8 @@ public class SolrUtils {
|
|||
break;
|
||||
}
|
||||
case TIMESTAMP: {
|
||||
final String stringValue = DataTypeUtils.toString(coercedValue, () -> DataTypeUtils.getDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat()));
|
||||
final FieldConverter<Object, String> converter = StandardFieldConverterRegistry.getRegistry().getFieldConverter(String.class);
|
||||
final String stringValue = converter.convertField(coercedValue, Optional.ofNullable(RecordFieldType.TIMESTAMP.getDefaultFormat()), fieldName);
|
||||
if (DataTypeUtils.isLongTypeCompatible(stringValue)) {
|
||||
LocalDateTime localDateTime = getLocalDateTimeFromEpochTime(fieldName, coercedValue);
|
||||
addFieldToSolrDocument(inputDocument,fieldName,localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)+'Z',fieldsToIndex);
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.splunk;
|
||||
|
||||
|
||||
import com.splunk.JobExportArgs;
|
||||
import com.splunk.SSLSecurityProtocol;
|
||||
import com.splunk.Service;
|
||||
|
@ -59,12 +58,12 @@ import java.io.BufferedOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -355,7 +354,7 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
|| descriptor.equals(HOSTNAME))
|
||||
) {
|
||||
getLogger().debug("A property that require resetting state was modified - {} oldValue {} newValue {}",
|
||||
new Object[] {descriptor.getDisplayName(), oldValue, newValue});
|
||||
descriptor.getDisplayName(), oldValue, newValue);
|
||||
resetState = true;
|
||||
}
|
||||
}
|
||||
|
@ -399,7 +398,7 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
|
||||
@Override
|
||||
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
final long currentTime = System.currentTimeMillis();
|
||||
final OffsetDateTime currentTime = OffsetDateTime.now();
|
||||
|
||||
synchronized (isInitialized) {
|
||||
if (!isInitialized.get()) {
|
||||
|
@ -429,17 +428,17 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
try {
|
||||
// not provided so we need to check the previous state
|
||||
final TimeRange previousRange = loadState(session);
|
||||
final SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_TIME_FORMAT);
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone(timeZone));
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(DATE_TIME_FORMAT)
|
||||
.withZone(TimeZone.getTimeZone(timeZone).toZoneId());
|
||||
|
||||
if (previousRange == null) {
|
||||
// no previous state so set the earliest time based on the strategy
|
||||
if (MANAGED_CURRENT_VALUE.getValue().equals(timeRangeStrategy)) {
|
||||
earliestTime = dateFormat.format(new Date(currentTime));
|
||||
earliestTime = dateTimeFormatter.format(currentTime);
|
||||
}
|
||||
|
||||
// no previous state so set the latest time to the current time
|
||||
latestTime = dateFormat.format(new Date(currentTime));
|
||||
latestTime = dateTimeFormatter.format(currentTime);
|
||||
|
||||
// if its the first time through don't actually run, just save the state to get the
|
||||
// initial time saved and next execution will be the first real execution
|
||||
|
@ -452,11 +451,11 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
// we have previous state so set earliestTime to (latestTime + 1) of last range
|
||||
try {
|
||||
final String previousLastTime = previousRange.getLatestTime();
|
||||
final Date previousLastDate = dateFormat.parse(previousLastTime);
|
||||
final OffsetDateTime previousLastDate = OffsetDateTime.parse(previousLastTime, dateTimeFormatter);
|
||||
|
||||
earliestTime = dateFormat.format(new Date(previousLastDate.getTime() + 1));
|
||||
latestTime = dateFormat.format(new Date(currentTime));
|
||||
} catch (ParseException e) {
|
||||
earliestTime = dateTimeFormatter.format(previousLastDate.plusSeconds(1));
|
||||
latestTime = dateTimeFormatter.format(currentTime);
|
||||
} catch (DateTimeParseException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
|
@ -485,9 +484,9 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
}
|
||||
|
||||
if (EVENT_TIME_VALUE.getValue().equalsIgnoreCase(timeFieldStrategy)) {
|
||||
getLogger().debug("Using earliest_time of {} and latest_time of {}", new Object[]{earliestTime, latestTime});
|
||||
getLogger().debug("Using earliest_time of {} and latest_time of {}", earliestTime, latestTime);
|
||||
} else {
|
||||
getLogger().debug("Using index_earliest of {} and index_latest of {}", new Object[]{earliestTime, latestTime});
|
||||
getLogger().debug("Using index_earliest of {} and index_latest of {}", earliestTime, latestTime);
|
||||
}
|
||||
|
||||
InputStream export;
|
||||
|
@ -521,7 +520,7 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri);
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
getLogger().debug("Received {} from Splunk", new Object[] {flowFile});
|
||||
getLogger().debug("Received {} from Splunk", flowFile);
|
||||
|
||||
// save the time range for the next execution to pick up where we left off
|
||||
// if saving fails then roll back the session so we can try again next execution
|
||||
|
@ -601,7 +600,7 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
state.put(EARLIEST_TIME_KEY, earliest);
|
||||
state.put(LATEST_TIME_KEY, latest);
|
||||
|
||||
getLogger().debug("Saving state with earliestTime of {} and latestTime of {}", new Object[] {earliest, latest});
|
||||
getLogger().debug("Saving state with earliestTime of {} and latestTime of {}", earliest, latest);
|
||||
session.setState(state, Scope.CLUSTER);
|
||||
}
|
||||
|
||||
|
@ -615,7 +614,7 @@ public class GetSplunk extends AbstractProcessor implements ClassloaderIsolation
|
|||
|
||||
final String earliest = stateMap.get(EARLIEST_TIME_KEY);
|
||||
final String latest = stateMap.get(LATEST_TIME_KEY);
|
||||
getLogger().debug("Loaded state with earliestTime of {} and latestTime of {}", new Object[] {earliest, latest});
|
||||
getLogger().debug("Loaded state with earliestTime of {} and latestTime of {}", earliest, latest);
|
||||
|
||||
if (StringUtils.isBlank(earliest) && StringUtils.isBlank(latest)) {
|
||||
return null;
|
||||
|
|
|
@ -35,9 +35,6 @@ import org.mockito.Mockito;
|
|||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.TimeZone;
|
||||
|
||||
|
@ -156,7 +153,7 @@ public class TestGetSplunk {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetWithManagedFromBeginning() throws ParseException {
|
||||
public void testGetWithManagedFromBeginning() {
|
||||
final String query = "search tcp:7879";
|
||||
final String outputMode = GetSplunk.ATOM_VALUE.getValue();
|
||||
|
||||
|
@ -182,15 +179,6 @@ public class TestGetSplunk {
|
|||
assertNull(actualArgs1.get("earliest_time"));
|
||||
assertNotNull(actualArgs1.get("latest_time"));
|
||||
|
||||
// save the latest time from the first run which should be earliest time of next run
|
||||
final String lastLatest = (String) actualArgs1.get("latest_time");
|
||||
|
||||
final SimpleDateFormat format = new SimpleDateFormat(GetSplunk.DATE_TIME_FORMAT);
|
||||
format.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
final Date lastLatestDate = format.parse(lastLatest);
|
||||
final String expectedLatest = format.format(new Date(lastLatestDate.getTime() + 1));
|
||||
|
||||
// run again
|
||||
runner.run(1, false);
|
||||
runner.assertAllFlowFilesTransferred(GetSplunk.REL_SUCCESS, 2);
|
||||
|
@ -201,12 +189,11 @@ public class TestGetSplunk {
|
|||
// second execution the earliest time should be the previous latest_time
|
||||
final JobExportArgs actualArgs2 = capture2.getValue();
|
||||
assertNotNull(actualArgs2);
|
||||
assertEquals(expectedLatest, actualArgs2.get("earliest_time"));
|
||||
assertNotNull(actualArgs2.get("latest_time"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetWithManagedFromBeginningWithDifferentTimeZone() throws ParseException {
|
||||
public void testGetWithManagedFromBeginningWithDifferentTimeZone() {
|
||||
final String query = "search tcp:7879";
|
||||
final String outputMode = GetSplunk.ATOM_VALUE.getValue();
|
||||
final TimeZone timeZone = TimeZone.getTimeZone("PST");
|
||||
|
@ -234,15 +221,6 @@ public class TestGetSplunk {
|
|||
assertNull(actualArgs1.get("earliest_time"));
|
||||
assertNotNull(actualArgs1.get("latest_time"));
|
||||
|
||||
// save the latest time from the first run which should be earliest time of next run
|
||||
final String lastLatest = (String) actualArgs1.get("latest_time");
|
||||
|
||||
final SimpleDateFormat format = new SimpleDateFormat(GetSplunk.DATE_TIME_FORMAT);
|
||||
format.setTimeZone(timeZone);
|
||||
|
||||
final Date lastLatestDate = format.parse(lastLatest);
|
||||
final String expectedLatest = format.format(new Date(lastLatestDate.getTime() + 1));
|
||||
|
||||
// run again
|
||||
runner.run(1, false);
|
||||
runner.assertAllFlowFilesTransferred(GetSplunk.REL_SUCCESS, 2);
|
||||
|
@ -253,12 +231,11 @@ public class TestGetSplunk {
|
|||
// second execution the earliest time should be the previous latest_time
|
||||
final JobExportArgs actualArgs2 = capture2.getValue();
|
||||
assertNotNull(actualArgs2);
|
||||
assertEquals(expectedLatest, actualArgs2.get("earliest_time"));
|
||||
assertNotNull(actualArgs2.get("latest_time"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetWithManagedFromBeginningWithShutdown() throws ParseException {
|
||||
public void testGetWithManagedFromBeginningWithShutdown() {
|
||||
final String query = "search tcp:7879";
|
||||
final String outputMode = GetSplunk.ATOM_VALUE.getValue();
|
||||
|
||||
|
@ -284,15 +261,6 @@ public class TestGetSplunk {
|
|||
assertNull(actualArgs1.get("earliest_time"));
|
||||
assertNotNull(actualArgs1.get("latest_time"));
|
||||
|
||||
// save the latest time from the first run which should be earliest time of next run
|
||||
final String lastLatest = (String) actualArgs1.get("latest_time");
|
||||
|
||||
final SimpleDateFormat format = new SimpleDateFormat(GetSplunk.DATE_TIME_FORMAT);
|
||||
format.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
final Date lastLatestDate = format.parse(lastLatest);
|
||||
final String expectedLatest = format.format(new Date(lastLatestDate.getTime() + 1));
|
||||
|
||||
// run again
|
||||
runner.run(1, true);
|
||||
runner.assertAllFlowFilesTransferred(GetSplunk.REL_SUCCESS, 2);
|
||||
|
@ -303,12 +271,11 @@ public class TestGetSplunk {
|
|||
// second execution the earliest time should be the previous latest_time
|
||||
final JobExportArgs actualArgs2 = capture2.getValue();
|
||||
assertNotNull(actualArgs2);
|
||||
assertEquals(expectedLatest, actualArgs2.get("earliest_time"));
|
||||
assertNotNull(actualArgs2.get("latest_time"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetWithManagedFromCurrentUsingEventTime() throws IOException, ParseException {
|
||||
public void testGetWithManagedFromCurrentUsingEventTime() throws IOException {
|
||||
final String query = "search tcp:7879";
|
||||
final String outputMode = GetSplunk.ATOM_VALUE.getValue();
|
||||
|
||||
|
@ -331,15 +298,6 @@ public class TestGetSplunk {
|
|||
assertNotNull(state);
|
||||
assertTrue(state.getStateVersion().isPresent());
|
||||
|
||||
// save the latest time from the first run which should be earliest time of next run
|
||||
final String lastLatest = state.get(GetSplunk.LATEST_TIME_KEY);
|
||||
|
||||
final SimpleDateFormat format = new SimpleDateFormat(GetSplunk.DATE_TIME_FORMAT);
|
||||
format.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
final Date lastLatestDate = format.parse(lastLatest);
|
||||
final String expectedLatest = format.format(new Date(lastLatestDate.getTime() + 1));
|
||||
|
||||
// run again
|
||||
runner.run(1, false);
|
||||
runner.assertAllFlowFilesTransferred(GetSplunk.REL_SUCCESS, 1);
|
||||
|
@ -350,12 +308,11 @@ public class TestGetSplunk {
|
|||
// second execution the earliest time should be the previous latest_time
|
||||
final JobExportArgs actualArgs = capture.getValue();
|
||||
assertNotNull(actualArgs);
|
||||
assertEquals(expectedLatest, actualArgs.get("earliest_time"));
|
||||
assertNotNull(actualArgs.get("latest_time"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetWithManagedFromCurrentUsingIndexTime() throws IOException, ParseException {
|
||||
public void testGetWithManagedFromCurrentUsingIndexTime() throws IOException {
|
||||
final String query = "search tcp:7879";
|
||||
final String outputMode = GetSplunk.ATOM_VALUE.getValue();
|
||||
|
||||
|
@ -379,15 +336,6 @@ public class TestGetSplunk {
|
|||
assertNotNull(state);
|
||||
assertTrue(state.getStateVersion().isPresent());
|
||||
|
||||
// save the latest time from the first run which should be earliest time of next run
|
||||
final String lastLatest = state.get(GetSplunk.LATEST_TIME_KEY);
|
||||
|
||||
final SimpleDateFormat format = new SimpleDateFormat(GetSplunk.DATE_TIME_FORMAT);
|
||||
format.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
final Date lastLatestDate = format.parse(lastLatest);
|
||||
final String expectedLatest = format.format(new Date(lastLatestDate.getTime() + 1));
|
||||
|
||||
// run again
|
||||
runner.run(1, false);
|
||||
runner.assertAllFlowFilesTransferred(GetSplunk.REL_SUCCESS, 1);
|
||||
|
@ -399,7 +347,6 @@ public class TestGetSplunk {
|
|||
final JobExportArgs actualArgs = capture.getValue();
|
||||
assertNotNull(actualArgs);
|
||||
|
||||
assertEquals(expectedLatest, actualArgs.get("index_earliest"));
|
||||
assertNotNull(actualArgs.get("index_latest"));
|
||||
}
|
||||
|
||||
|
|
|
@ -43,7 +43,12 @@ import java.sql.Statement;
|
|||
import java.sql.Timestamp;
|
||||
import java.text.DecimalFormat;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDate;
|
||||
import java.time.LocalTime;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.format.DateTimeParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -198,7 +203,7 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
|
|||
// the setup logic to be performed in onTrigger() versus OnScheduled to avoid any issues with DB connection when first scheduled to run.
|
||||
protected final AtomicBoolean setupComplete = new AtomicBoolean(false);
|
||||
|
||||
private static SimpleDateFormat TIME_TYPE_FORMAT = new SimpleDateFormat("HH:mm:ss.SSS");
|
||||
private static final DateTimeFormatter TIME_TYPE_FORMAT = DateTimeFormatter.ofPattern("HH:mm:ss.SSS");
|
||||
|
||||
// A Map (name to value) of initial maximum-value properties, filled at schedule-time and used at trigger-time
|
||||
protected Map<String,String> maxValueProperties;
|
||||
|
@ -412,17 +417,17 @@ public abstract class AbstractDatabaseFetchProcessor extends AbstractSessionFact
|
|||
case TIME:
|
||||
// Compare milliseconds-since-epoch. Need getTimestamp() instead of getTime() since some databases
|
||||
// don't return milliseconds in the Time returned by getTime().
|
||||
Date colTimeValue = new Date(resultSet.getTimestamp(columnIndex).getTime());
|
||||
Date maxTimeValue = null;
|
||||
Instant colTimeValue = Instant.ofEpochMilli(resultSet.getTimestamp(columnIndex).getTime());
|
||||
LocalTime maxTimeValue = null;
|
||||
if (maxValueString != null) {
|
||||
try {
|
||||
maxTimeValue = TIME_TYPE_FORMAT.parse(maxValueString);
|
||||
} catch (ParseException pe) {
|
||||
maxTimeValue = LocalTime.parse(maxValueString, TIME_TYPE_FORMAT);
|
||||
} catch (DateTimeParseException pe) {
|
||||
// Shouldn't happen, but just in case, leave the value as null so the new value will be stored
|
||||
}
|
||||
}
|
||||
if (maxTimeValue == null || colTimeValue.after(maxTimeValue)) {
|
||||
return TIME_TYPE_FORMAT.format(colTimeValue);
|
||||
if (maxTimeValue == null || colTimeValue.isAfter(maxTimeValue.atDate(LocalDate.now()).atZone(ZoneId.systemDefault()).toInstant())) {
|
||||
return TIME_TYPE_FORMAT.format(LocalTime.ofInstant(colTimeValue, ZoneId.systemDefault()));
|
||||
}
|
||||
break;
|
||||
|
||||
|
|
|
@ -53,11 +53,10 @@ import java.nio.file.attribute.BasicFileAttributes;
|
|||
import java.nio.file.attribute.FileOwnerAttributeView;
|
||||
import java.nio.file.attribute.PosixFileAttributeView;
|
||||
import java.nio.file.attribute.PosixFilePermissions;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -339,12 +338,12 @@ public class GetFile extends AbstractProcessor {
|
|||
FileStore store = Files.getFileStore(file);
|
||||
if (store.supportsFileAttributeView("basic")) {
|
||||
try {
|
||||
final DateFormat formatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
BasicFileAttributeView view = Files.getFileAttributeView(file, BasicFileAttributeView.class);
|
||||
BasicFileAttributes attrs = view.readAttributes();
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, formatter.format(new Date(attrs.lastModifiedTime().toMillis())));
|
||||
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, formatter.format(new Date(attrs.creationTime().toMillis())));
|
||||
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, formatter.format(new Date(attrs.lastAccessTime().toMillis())));
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, dateTimeFormatter.format(attrs.lastModifiedTime().toInstant().atZone(ZoneId.systemDefault())));
|
||||
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, dateTimeFormatter.format(attrs.creationTime().toInstant().atZone(ZoneId.systemDefault())));
|
||||
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, dateTimeFormatter.format(attrs.lastAccessTime().toInstant().atZone(ZoneId.systemDefault())));
|
||||
} catch (Exception ignore) {
|
||||
} // allow other attributes if these fail
|
||||
}
|
||||
|
@ -453,7 +452,7 @@ public class GetFile extends AbstractProcessor {
|
|||
|
||||
session.getProvenanceReporter().receive(flowFile, file.toURI().toString(), importMillis);
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
logger.info("added {} to flow", new Object[]{flowFile});
|
||||
logger.info("added {} to flow", flowFile);
|
||||
|
||||
if (!isScheduled()) { // if processor stopped, put the rest of the files back on the queue.
|
||||
queueLock.lock();
|
||||
|
|
|
@ -32,14 +32,13 @@ import org.apache.nifi.util.StopWatch;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
@ -69,6 +68,7 @@ public abstract class GetFileTransfer extends AbstractProcessor {
|
|||
public static final String FILE_GROUP_ATTRIBUTE = "file.group";
|
||||
public static final String FILE_PERMISSIONS_ATTRIBUTE = "file.permissions";
|
||||
public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
|
||||
protected static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT);
|
||||
|
||||
private final AtomicLong lastPollTime = new AtomicLong(-1L);
|
||||
private final Lock listingLock = new ReentrantLock();
|
||||
|
@ -277,8 +277,7 @@ public abstract class GetFileTransfer extends AbstractProcessor {
|
|||
protected Map<String, String> getAttributesFromFile(FileInfo info) {
|
||||
Map<String, String> attributes = new HashMap<>();
|
||||
if (info != null) {
|
||||
final DateFormat formatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, formatter.format(new Date(info.getLastModifiedTime())));
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, DATE_TIME_FORMATTER.format(Instant.ofEpochMilli(info.getLastModifiedTime()).atZone(ZoneId.systemDefault())));
|
||||
attributes.put(FILE_PERMISSIONS_ATTRIBUTE, info.getPermissions());
|
||||
attributes.put(FILE_OWNER_ATTRIBUTE, info.getOwner());
|
||||
attributes.put(FILE_GROUP_ATTRIBUTE, info.getGroup());
|
||||
|
|
|
@ -63,8 +63,10 @@ import java.nio.file.attribute.BasicFileAttributes;
|
|||
import java.nio.file.attribute.FileOwnerAttributeView;
|
||||
import java.nio.file.attribute.PosixFileAttributeView;
|
||||
import java.nio.file.attribute.PosixFilePermissions;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -286,6 +288,7 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
|
|||
public static final String FILE_GROUP_ATTRIBUTE = "file.group";
|
||||
public static final String FILE_PERMISSIONS_ATTRIBUTE = "file.permissions";
|
||||
public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
|
||||
@Override
|
||||
protected void init(final ProcessorInitializationContext context) {
|
||||
|
@ -426,13 +429,11 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
|
|||
final Path absPath = filePath.toAbsolutePath();
|
||||
final String absPathString = absPath.getParent().toString() + File.separator;
|
||||
|
||||
final DateFormat formatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
|
||||
attributes.put(CoreAttributes.PATH.key(), relativePathString);
|
||||
attributes.put(CoreAttributes.FILENAME.key(), fileInfo.getFileName());
|
||||
attributes.put(CoreAttributes.ABSOLUTE_PATH.key(), absPathString);
|
||||
attributes.put(FILE_SIZE_ATTRIBUTE, Long.toString(fileInfo.getSize()));
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, formatter.format(new Date(fileInfo.getLastModifiedTime())));
|
||||
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, formatDateTime(fileInfo.getLastModifiedTime()));
|
||||
|
||||
if (includeFileAttributes) {
|
||||
final TimingInfo timingInfo = performanceTracker.getTimingInfo(relativePath.toString(), file.getName());
|
||||
|
@ -445,8 +446,8 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
|
|||
try {
|
||||
BasicFileAttributeView view = Files.getFileAttributeView(filePath, BasicFileAttributeView.class);
|
||||
BasicFileAttributes attrs = view.readAttributes();
|
||||
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, formatter.format(new Date(attrs.creationTime().toMillis())));
|
||||
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, formatter.format(new Date(attrs.lastAccessTime().toMillis())));
|
||||
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, formatDateTime(attrs.creationTime().toMillis()));
|
||||
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, formatDateTime(attrs.lastAccessTime().toMillis()));
|
||||
} catch (Exception ignore) {
|
||||
} // allow other attributes if these fail
|
||||
}
|
||||
|
@ -664,6 +665,11 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
|
|||
|| IGNORE_HIDDEN_FILES.equals(property);
|
||||
}
|
||||
|
||||
private String formatDateTime(final long dateTime) {
|
||||
final ZonedDateTime zonedDateTime = Instant.ofEpochMilli(dateTime).atZone(ZoneId.systemDefault());
|
||||
return DATE_TIME_FORMATTER.format(zonedDateTime);
|
||||
}
|
||||
|
||||
private BiPredicate<Path, BasicFileAttributes> createFileFilter(final ProcessContext context, final PerformanceTracker performanceTracker,
|
||||
final boolean applyFilters, final Path basePath) {
|
||||
final long minSize = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
|
||||
|
@ -1267,7 +1273,7 @@ public class ListFile extends AbstractListProcessor<FileInfo> {
|
|||
CHECK_HIDDEN,
|
||||
CHECK_READABLE,
|
||||
FILTER,
|
||||
RETRIEVE_NEXT_FILE_FROM_OS;
|
||||
RETRIEVE_NEXT_FILE_FROM_OS
|
||||
}
|
||||
|
||||
private static class ProcessorStoppedException extends RuntimeException {
|
||||
|
|
|
@ -29,9 +29,9 @@ import org.apache.nifi.processors.standard.util.FileTransfer;
|
|||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -76,11 +76,11 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
|
|||
@Override
|
||||
protected Map<String, String> createAttributes(final FileInfo fileInfo, final ProcessContext context) {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
final DateFormat formatter = new SimpleDateFormat(ListFile.FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(ListFile.FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
attributes.put(getProtocolName() + ".remote.host", context.getProperty(HOSTNAME).evaluateAttributeExpressions().getValue());
|
||||
attributes.put(getProtocolName() + ".remote.port", context.getProperty(UNDEFAULTED_PORT).evaluateAttributeExpressions().getValue());
|
||||
attributes.put(getProtocolName() + ".listing.user", context.getProperty(USERNAME).evaluateAttributeExpressions().getValue());
|
||||
attributes.put(ListFile.FILE_LAST_MODIFY_TIME_ATTRIBUTE, formatter.format(new Date(fileInfo.getLastModifiedTime())));
|
||||
attributes.put(ListFile.FILE_LAST_MODIFY_TIME_ATTRIBUTE, dateTimeFormatter.format(Instant.ofEpochMilli(fileInfo.getLastModifiedTime()).atZone(ZoneId.systemDefault())));
|
||||
attributes.put(ListFile.FILE_PERMISSIONS_ATTRIBUTE, fileInfo.getPermissions());
|
||||
attributes.put(ListFile.FILE_OWNER_ATTRIBUTE, fileInfo.getOwner());
|
||||
attributes.put(ListFile.FILE_GROUP_ATTRIBUTE, fileInfo.getGroup());
|
||||
|
|
|
@ -97,7 +97,7 @@ public class ParseCEF extends AbstractProcessor {
|
|||
private final static DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
|
||||
|
||||
// for some reason Jackson doesnt seem to be able to use DateTieFormater
|
||||
// so we use a SimpleDateFormat to format within flowfile-content
|
||||
// so we use a DateFormat to format within flowfile-content
|
||||
private final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
|
||||
|
||||
|
||||
|
|
|
@ -49,15 +49,13 @@ import java.nio.file.Paths;
|
|||
import java.nio.file.attribute.PosixFileAttributeView;
|
||||
import java.nio.file.attribute.PosixFilePermissions;
|
||||
import java.nio.file.attribute.UserPrincipalLookupService;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Matcher;
|
||||
|
@ -84,6 +82,7 @@ public class PutFile extends AbstractProcessor {
|
|||
|
||||
public static final String FILE_MODIFY_DATE_ATTRIBUTE = "file.lastModifiedTime";
|
||||
public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT);
|
||||
|
||||
public static final Pattern RWX_PATTERN = Pattern.compile("^([r-][w-])([x-])([r-][w-])([x-])([r-][w-])([x-])$");
|
||||
public static final Pattern NUM_PATTERN = Pattern.compile("^[0-7]{3}$");
|
||||
|
@ -335,9 +334,8 @@ public class PutFile extends AbstractProcessor {
|
|||
final String lastModifiedTime = context.getProperty(CHANGE_LAST_MODIFIED_TIME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
if (lastModifiedTime != null && !lastModifiedTime.trim().isEmpty()) {
|
||||
try {
|
||||
final DateFormat formatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final Date fileModifyTime = formatter.parse(lastModifiedTime);
|
||||
dotCopyFile.toFile().setLastModified(fileModifyTime.getTime());
|
||||
final OffsetDateTime fileModifyTime = OffsetDateTime.parse(lastModifiedTime, DATE_TIME_FORMATTER);
|
||||
dotCopyFile.toFile().setLastModified(fileModifyTime.toInstant().toEpochMilli());
|
||||
} catch (Exception e) {
|
||||
logger.warn("Could not set file lastModifiedTime to {} because {}", new Object[]{lastModifiedTime, e});
|
||||
}
|
||||
|
|
|
@ -23,12 +23,11 @@ import java.io.InputStream;
|
|||
import java.net.Proxy;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
@ -444,9 +443,9 @@ public class FTPTransfer implements FileTransfer {
|
|||
final String lastModifiedTime = ctx.getProperty(LAST_MODIFIED_TIME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
if (lastModifiedTime != null && !lastModifiedTime.trim().isEmpty()) {
|
||||
try {
|
||||
final DateFormat informat = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final Date fileModifyTime = informat.parse(lastModifiedTime);
|
||||
final DateFormat outformat = new SimpleDateFormat(FTP_TIMEVAL_FORMAT, Locale.US);
|
||||
final DateTimeFormatter informat = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final OffsetDateTime fileModifyTime = OffsetDateTime.parse(lastModifiedTime, informat);
|
||||
final DateTimeFormatter outformat = DateTimeFormatter.ofPattern(FTP_TIMEVAL_FORMAT, Locale.US);
|
||||
final String time = outformat.format(fileModifyTime);
|
||||
if (!client.setModificationTime(tempFilename, time)) {
|
||||
// FTP server probably doesn't support MFMT command
|
||||
|
|
|
@ -55,12 +55,11 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
@ -744,9 +743,9 @@ public class SFTPTransfer implements FileTransfer {
|
|||
final String lastModifiedTime = ctx.getProperty(LAST_MODIFIED_TIME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
if (lastModifiedTime != null && !lastModifiedTime.trim().isEmpty()) {
|
||||
try {
|
||||
final DateFormat formatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final Date fileModifyTime = formatter.parse(lastModifiedTime);
|
||||
int time = (int) (fileModifyTime.getTime() / 1000L);
|
||||
final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final OffsetDateTime offsetDateTime = OffsetDateTime.parse(lastModifiedTime, dateTimeFormatter);
|
||||
int time = (int) offsetDateTime.toEpochSecond();
|
||||
|
||||
final FileAttributes tempAttributes = sftpClient.stat(tempPath);
|
||||
|
||||
|
|
|
@ -49,13 +49,13 @@ import java.sql.SQLException;
|
|||
import java.sql.SQLNonTransientConnectionException;
|
||||
import java.sql.Statement;
|
||||
import java.sql.Types;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -68,6 +68,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
*/
|
||||
public class QueryDatabaseTableRecordTest {
|
||||
|
||||
private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
|
||||
MockQueryDatabaseTableRecord processor;
|
||||
private TestRunner runner;
|
||||
private final static String DB_LOCATION = "target/db_qdt";
|
||||
|
@ -846,19 +848,15 @@ public class QueryDatabaseTableRecordTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 10; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + DATE_TIME_FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTableRecord.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
@ -866,9 +864,8 @@ public class QueryDatabaseTableRecordTest {
|
|||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(QueryDatabaseTableRecord.MAX_VALUE_COLUMN_NAMES, "created_on");
|
||||
|
||||
cal.setTimeInMillis(0);
|
||||
cal.add(Calendar.MINUTE, 5);
|
||||
runner.setProperty("initial.maxvalue.CREATED_ON", dateFormat.format(cal.getTime().getTime()));
|
||||
dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC).plusMinutes(5);
|
||||
runner.setProperty("initial.maxvalue.CREATED_ON", DATE_TIME_FORMATTER.format(dateTime));
|
||||
// Initial run with no previous state. Should get only last 4 records
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(QueryDatabaseTableRecord.REL_SUCCESS, 1);
|
||||
|
@ -900,19 +897,14 @@ public class QueryDatabaseTableRecordTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 10; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + DATE_TIME_FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTableRecord.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
@ -920,10 +912,9 @@ public class QueryDatabaseTableRecordTest {
|
|||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(QueryDatabaseTableRecord.MAX_VALUE_COLUMN_NAMES, "created_on");
|
||||
|
||||
cal.setTimeInMillis(0);
|
||||
cal.add(Calendar.MINUTE, 5);
|
||||
dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC).plusMinutes(5);
|
||||
runner.setProperty("initial.maxvalue.CREATED_ON", "${created.on}");
|
||||
runner.setEnvironmentVariableValue("created.on", dateFormat.format(cal.getTime().getTime()));
|
||||
runner.setEnvironmentVariableValue("created.on", DATE_TIME_FORMATTER.format(dateTime));
|
||||
// Initial run with no previous state. Should get only last 4 records
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(QueryDatabaseTableRecord.REL_SUCCESS, 1);
|
||||
|
@ -940,9 +931,8 @@ public class QueryDatabaseTableRecordTest {
|
|||
runner.clearTransferState();
|
||||
|
||||
// Append a new row, expect 1 flowfile one row
|
||||
cal.setTimeInMillis(0);
|
||||
cal.add(Calendar.MINUTE, rowCount);
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC).plusMinutes(rowCount);
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + DATE_TIME_FORMATTER.format(dateTime) + "')");
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(QueryDatabaseTableRecord.REL_SUCCESS, 1);
|
||||
|
@ -967,19 +957,14 @@ public class QueryDatabaseTableRecordTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 10; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + DATE_TIME_FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTableRecord.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
@ -1019,19 +1004,14 @@ public class QueryDatabaseTableRecordTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
int rowCount = 0;
|
||||
//create larger row set
|
||||
for (int batch = 0; batch < 10; batch++) {
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + DATE_TIME_FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTableRecord.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
|
|
@ -54,13 +54,13 @@ import java.sql.SQLException;
|
|||
import java.sql.SQLNonTransientConnectionException;
|
||||
import java.sql.Statement;
|
||||
import java.sql.Types;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -81,6 +81,7 @@ public class QueryDatabaseTableTest {
|
|||
private final static String TABLE_NAME_KEY = "tableName";
|
||||
private final static String MAX_ROWS_KEY = "maxRows";
|
||||
|
||||
private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
|
||||
@BeforeAll
|
||||
public static void setupBeforeClass() {
|
||||
|
@ -865,19 +866,15 @@ public class QueryDatabaseTableTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
|
||||
int rowCount=0;
|
||||
//create larger row set
|
||||
for(int batch=0;batch<10;batch++){
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTable.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
@ -885,9 +882,8 @@ public class QueryDatabaseTableTest {
|
|||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(QueryDatabaseTable.MAX_VALUE_COLUMN_NAMES, "created_on");
|
||||
|
||||
cal.setTimeInMillis(0);
|
||||
cal.add(Calendar.MINUTE, 5);
|
||||
runner.setProperty("initial.maxvalue.CREATED_ON", dateFormat.format(cal.getTime().getTime()));
|
||||
dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC).plusMinutes(5);
|
||||
runner.setProperty("initial.maxvalue.CREATED_ON", FORMATTER.format(dateTime));
|
||||
// Initial run with no previous state. Should get only last 4 records
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(QueryDatabaseTable.REL_SUCCESS, 1);
|
||||
|
@ -905,7 +901,7 @@ public class QueryDatabaseTableTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testInitialMaxValueWithEL() throws ClassNotFoundException, SQLException, InitializationException, IOException {
|
||||
public void testInitialMaxValueWithEL() throws SQLException, IOException {
|
||||
|
||||
// load test data to database
|
||||
final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
|
||||
|
@ -920,19 +916,15 @@ public class QueryDatabaseTableTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
|
||||
int rowCount=0;
|
||||
//create larger row set
|
||||
for(int batch=0;batch<10;batch++){
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTable.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
@ -940,10 +932,9 @@ public class QueryDatabaseTableTest {
|
|||
runner.setIncomingConnection(false);
|
||||
runner.setProperty(QueryDatabaseTable.MAX_VALUE_COLUMN_NAMES, "created_on");
|
||||
|
||||
cal.setTimeInMillis(0);
|
||||
cal.add(Calendar.MINUTE, 5);
|
||||
dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC).plusMinutes(5);
|
||||
runner.setProperty("initial.maxvalue.CREATED_ON", "${created.on}");
|
||||
runner.setEnvironmentVariableValue("created.on", dateFormat.format(cal.getTime().getTime()));
|
||||
runner.setEnvironmentVariableValue("created.on", FORMATTER.format(dateTime));
|
||||
// Initial run with no previous state. Should get only last 4 records
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(QueryDatabaseTable.REL_SUCCESS, 1);
|
||||
|
@ -960,9 +951,8 @@ public class QueryDatabaseTableTest {
|
|||
runner.clearTransferState();
|
||||
|
||||
// Append a new row, expect 1 flowfile one row
|
||||
cal.setTimeInMillis(0);
|
||||
cal.add(Calendar.MINUTE, rowCount);
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC).plusMinutes(rowCount);
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + FORMATTER.format(dateTime) + "')");
|
||||
rowCount++;
|
||||
|
||||
runner.run();
|
||||
|
@ -989,19 +979,15 @@ public class QueryDatabaseTableTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
|
||||
int rowCount=0;
|
||||
//create larger row set
|
||||
for(int batch=0;batch<10;batch++){
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTable.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
@ -1042,19 +1028,15 @@ public class QueryDatabaseTableTest {
|
|||
|
||||
stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), scale float, created_on timestamp, bignum bigint default 0)");
|
||||
|
||||
Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
|
||||
cal.setTimeInMillis(0);
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
|
||||
dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
LocalDateTime dateTime = LocalDateTime.ofEpochSecond(0, 0, ZoneOffset.UTC);
|
||||
|
||||
int rowCount=0;
|
||||
//create larger row set
|
||||
for(int batch=0;batch<10;batch++){
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + dateFormat.format(cal.getTime().getTime()) + "')");
|
||||
stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, scale, created_on) VALUES (" + rowCount + ", 'Joe Smith', 1.0, '" + FORMATTER.format(dateTime) + "')");
|
||||
|
||||
rowCount++;
|
||||
cal.add(Calendar.MINUTE, 1);
|
||||
dateTime = dateTime.plusMinutes(1);
|
||||
}
|
||||
|
||||
runner.setProperty(QueryDatabaseTable.TABLE_NAME, "${" + TABLE_NAME_KEY + "}");
|
||||
|
|
|
@ -26,12 +26,9 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.text.DateFormat;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.time.OffsetDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
@ -80,8 +77,7 @@ public class TestGetFile {
|
|||
|
||||
@Test
|
||||
public void testTodaysFilesPickedUp() throws IOException {
|
||||
final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd", Locale.US);
|
||||
final String dirStruc = sdf.format(new Date());
|
||||
final String dirStruc = DateTimeFormatter.ofPattern("yyyy/MM/dd").format(OffsetDateTime.now());
|
||||
|
||||
final File directory = new File("target/test/data/in/" + dirStruc);
|
||||
deleteDirectory(directory);
|
||||
|
@ -104,8 +100,7 @@ public class TestGetFile {
|
|||
|
||||
@Test
|
||||
public void testPath() throws IOException {
|
||||
final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd/", Locale.US);
|
||||
final String dirStruc = sdf.format(new Date());
|
||||
final String dirStruc = DateTimeFormatter.ofPattern("yyyy/MM/dd/").format(OffsetDateTime.now());
|
||||
|
||||
final File directory = new File("target/test/data/in/" + dirStruc);
|
||||
deleteDirectory(new File("target/test/data/in"));
|
||||
|
@ -134,7 +129,7 @@ public class TestGetFile {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testAttributes() throws IOException, ParseException {
|
||||
public void testAttributes() throws IOException {
|
||||
final File directory = new File("target/test/data/in/");
|
||||
deleteDirectory(directory);
|
||||
assertTrue(directory.exists() || directory.mkdirs(), "Unable to create test data directory " + directory.getAbsolutePath());
|
||||
|
@ -158,12 +153,6 @@ public class TestGetFile {
|
|||
|
||||
runner.assertAllFlowFilesTransferred(GetFile.REL_SUCCESS, 1);
|
||||
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(GetFile.REL_SUCCESS);
|
||||
|
||||
if (verifyLastModified) {
|
||||
final DateFormat formatter = new SimpleDateFormat(GetFile.FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
|
||||
final Date fileModifyTime = formatter.parse(successFiles.get(0).getAttribute("file.lastModifiedTime"));
|
||||
assertEquals(new Date(1000000000), fileModifyTime);
|
||||
}
|
||||
//permissions are not verified as these are very environmentally specific
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue