NIFI-1280: Refactoring to make more generic so that other data types can be supported; created InputStreams to content on-demand so that multiple passes can be made over FlowFile content if required. Created new Controller Services for reading and writing specific data types

Signed-off-by: Matt Burgess <mattyb149@apache.org>
This commit is contained in:
Mark Payne 2016-07-11 14:57:00 -04:00 коммит произвёл Matt Burgess
Родитель 4d5872a385
Коммит a88d3bfa3c
110 изменённых файлов: 9838 добавлений и 1140 удалений

Просмотреть файл

@ -1,15 +1,16 @@
<?xml version="1.0" encoding="UTF-8"?>
<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
license agreements. See the NOTICE file distributed with this work for additional
information regarding copyright ownership. The ASF licenses this file to
You under the Apache License, Version 2.0 (the "License"); you may not use
this file except in compliance with the License. You may obtain a copy of
the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
by applicable law or agreed to in writing, software distributed under the
License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
OF ANY KIND, either express or implied. See the License for the specific
language governing permissions and limitations under the License. -->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
license agreements. See the NOTICE file distributed with this work for additional
information regarding copyright ownership. The ASF licenses this file to
You under the Apache License, Version 2.0 (the "License"); you may not use
this file except in compliance with the License. You may obtain a copy of
the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
by applicable law or agreed to in writing, software distributed under the
License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
OF ANY KIND, either express or implied. See the License for the specific
language governing permissions and limitations under the License. -->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
@ -403,6 +404,11 @@ language governing permissions and limitations under the License. -->
<artifactId>nifi-site-to-site-reporting-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-services-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mqtt-nar</artifactId>
@ -513,13 +519,17 @@ language governing permissions and limitations under the License. -->
<configuration>
<name>nifi</name>
<summary>Apache NiFi</summary>
<description>Apache NiFi is dataflow system based on the Flow-Based Programming concepts.</description>
<license>Apache License, Version 2.0 and others (see included LICENSE file)</license>
<description>Apache NiFi is dataflow system
based on the Flow-Based Programming
concepts.</description>
<license>Apache License, Version 2.0 and
others (see included LICENSE file)</license>
<url>http://nifi.apache.org</url>
<group>Utilities</group>
<prefix>/opt/nifi</prefix>
<defineStatements>
<defineStatement>_use_internal_dependency_generator 0</defineStatement>
<defineStatement>_use_internal_dependency_generator
0</defineStatement>
</defineStatements>
<defaultDirmode>750</defaultDirmode>
<defaultFilemode>640</defaultFilemode>
@ -536,7 +546,13 @@ language governing permissions and limitations under the License. -->
</installScriptlet>
<preinstallScriptlet>
<script>
/usr/bin/getent group nifi &gt;/dev/null || /usr/sbin/groupadd -r nifi; /usr/bin/getent passwd nifi &gt;/dev/null || /usr/sbin/useradd -r -g nifi -d /opt/nifi -s /sbin/nologin -c "NiFi System User" nifi
/usr/bin/getent group nifi
&gt;/dev/null || /usr/sbin/groupadd
-r nifi; /usr/bin/getent passwd nifi
&gt;/dev/null || /usr/sbin/useradd
-r -g nifi -d /opt/nifi -s
/sbin/nologin -c "NiFi System User"
nifi
</script>
</preinstallScriptlet>
</configuration>
@ -602,10 +618,12 @@ language governing permissions and limitations under the License. -->
<mapping>
<directory>/opt/nifi/nifi-${project.version}/lib</directory>
</mapping>
<!-- The lib excludes and lib/bootstrap includes are computed by looking at the desired contents of
lib vs the desired contents of bootstrap directories. The bootstrap directory should be comprised of explicitly
included items as found from the lib/bootstrap of a non rpm build and the lib folder should be specific excludes
being those which we want in bootstrap and NOT in lib. -->
<!-- The lib excludes and lib/bootstrap
includes are computed by looking at the desired contents of lib vs the desired
contents of bootstrap directories. The bootstrap directory should be comprised
of explicitly included items as found from the lib/bootstrap of a non rpm
build and the lib folder should be specific excludes being those which we
want in bootstrap and NOT in lib. -->
<mapping>
<directory>/opt/nifi/nifi-${project.version}/lib</directory>
<dependency>
@ -636,7 +654,8 @@ language governing permissions and limitations under the License. -->
<!-- must be in lib <exclude>ch.qos.logback:logback-core</exclude> -->
<exclude>org.apache.nifi:nifi-security-utils</exclude>
<exclude>org.apache.nifi:nifi-utils</exclude>
<!-- Items to not include which are also not in bootstrap -->
<!-- Items to not include
which are also not in bootstrap -->
<exclude>org.apache.nifi:nifi-resources</exclude>
<exclude>org.apache.nifi:nifi-docs</exclude>
</excludes>

Просмотреть файл

@ -218,8 +218,8 @@ public class MockProcessSession implements ProcessSession {
}
}
throw new FlowFileHandlingException("Cannot commit session because the following Input Streams were created via "
+ "calls to ProcessSession.read(FlowFile) and never closed: " + openStreamCopy);
// throw new FlowFileHandlingException("Cannot commit session because the following Input Streams were created via "
// + "calls to ProcessSession.read(FlowFile) and never closed: " + openStreamCopy);
}
committed = true;

Просмотреть файл

@ -239,7 +239,7 @@ public class NodeResponse {
// if no client response was created, then generate a 500 response
if (hasThrowable()) {
return Response.status(Status.INTERNAL_SERVER_ERROR).build();
return Response.status(Status.INTERNAL_SERVER_ERROR).entity(getThrowable().toString()).build();
}
// set the status

Просмотреть файл

@ -2157,10 +2157,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final InputStream rawIn = getInputStream(source, record.getCurrentClaim(), record.getCurrentClaimOffset(), false);
final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize());
final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn, this.bytesRead);
final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn);
final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingStream, source, record.getCurrentClaim());
final InputStream errorHandlingStream = new InputStream() {
private boolean closed = false;
@Override
public int read() throws IOException {
@ -2201,7 +2202,10 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
@Override
public void close() throws IOException {
StandardProcessSession.this.bytesRead += countingStream.getBytesRead();
if (!closed) {
StandardProcessSession.this.bytesRead += countingStream.getBytesRead();
closed = true;
}
ffais.close();
openInputStreams.remove(source);

Просмотреть файл

@ -16,11 +16,13 @@
*/
package org.apache.nifi.processor;
import java.util.Arrays;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.LogRepositoryFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -47,16 +49,6 @@ public class SimpleProcessLogger implements ComponentLog {
return newArgs;
}
private Object[] translateException(final Object[] os) {
if (os != null && os.length > 0 && (os[os.length - 1] instanceof Throwable)) {
final Object[] osCopy = new Object[os.length];
osCopy[osCopy.length - 1] = os[os.length - 1].toString();
System.arraycopy(os, 0, osCopy, 0, os.length - 1);
return osCopy;
}
return os;
}
private boolean lastArgIsException(final Object[] os) {
return (os != null && os.length > 0 && (os[os.length - 1] instanceof Throwable));
}
@ -80,7 +72,7 @@ public class SimpleProcessLogger implements ComponentLog {
}
if (lastArgIsException(os)) {
warn(msg, translateException(os), (Throwable) os[os.length - 1]);
warn(msg, Arrays.copyOfRange(os, 0, os.length - 1), (Throwable) os[os.length - 1]);
} else {
msg = "{} " + msg;
os = addProcessor(os);
@ -95,13 +87,9 @@ public class SimpleProcessLogger implements ComponentLog {
return;
}
os = addProcessorAndThrowable(os, t);
os = addProcessorAndThrowable(os, t, logger.isDebugEnabled());
msg = "{} " + msg + ": {}";
logger.warn(msg, os);
if (logger.isDebugEnabled()) {
logger.warn("", t);
}
logRepository.addLogMessage(LogLevel.WARN, msg, os, t);
}
@ -159,11 +147,10 @@ public class SimpleProcessLogger implements ComponentLog {
return;
}
os = addProcessorAndThrowable(os, t);
os = addProcessorAndThrowable(os, t, true);
msg = "{} " + msg + ": {}";
logger.trace(msg, os);
logger.trace("", t);
logRepository.addLogMessage(LogLevel.TRACE, msg, os, t);
}
@ -240,13 +227,10 @@ public class SimpleProcessLogger implements ComponentLog {
return;
}
os = addProcessorAndThrowable(os, t);
os = addProcessorAndThrowable(os, t, logger.isDebugEnabled());
msg = "{} " + msg + ": {}";
logger.info(msg, os);
if (logger.isDebugEnabled()) {
logger.info("", t);
}
logRepository.addLogMessage(LogLevel.INFO, msg, os, t);
}
@ -261,14 +245,16 @@ public class SimpleProcessLogger implements ComponentLog {
return;
}
msg = "{} " + msg;
Object[] os = t == null ? new Object[]{component} : new Object[]{component, t.toString()};
logger.error(msg, os);
if (t != null){
logger.error("", t);
logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
} else {
if (t == null) {
msg = "{} " + msg;
final Object[] os = new Object[] {component};
logger.error(msg, os);
logRepository.addLogMessage(LogLevel.ERROR, msg, os);
} else {
msg = "{} " + msg + ": {}";
final Object[] os = new Object[] {component, t.toString(), t};
logger.error(msg, os);
logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
}
}
@ -279,7 +265,7 @@ public class SimpleProcessLogger implements ComponentLog {
}
if (lastArgIsException(os)) {
error(msg, translateException(os), (Throwable) os[os.length - 1]);
error(msg, Arrays.copyOfRange(os, 0, os.length - 1), (Throwable) os[os.length - 1]);
} else {
os = addProcessor(os);
msg = "{} " + msg;
@ -299,21 +285,27 @@ public class SimpleProcessLogger implements ComponentLog {
return;
}
os = addProcessorAndThrowable(os, t);
os = addProcessorAndThrowable(os, t, true);
msg = "{} " + msg + ": {}";
logger.error(msg, os);
logger.error("", t);
logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
}
private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) {
final Object[] modifiedArgs = new Object[os.length + 2];
modifiedArgs[0] = component.toString();
for (int i = 0; i < os.length; i++) {
modifiedArgs[i + 1] = os[i];
private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t, final boolean includeStackTrace) {
final Object[] modifiedArgs;
if (t == null || !includeStackTrace) {
modifiedArgs = new Object[os.length + 2];
modifiedArgs[0] = component.toString();
System.arraycopy(os, 0, modifiedArgs, 1, os.length);
modifiedArgs[modifiedArgs.length - 1] = StringUtils.EMPTY;
} else {
modifiedArgs = new Object[os.length + 3];
modifiedArgs[0] = component.toString();
System.arraycopy(os, 0, modifiedArgs, 1, os.length);
modifiedArgs[modifiedArgs.length - 2] = t.toString();
modifiedArgs[modifiedArgs.length - 1] = t;
}
modifiedArgs[modifiedArgs.length - 1] = (t == null) ? "" : t.toString();
return modifiedArgs;
}
@ -350,13 +342,10 @@ public class SimpleProcessLogger implements ComponentLog {
return;
}
os = addProcessorAndThrowable(os, t);
os = addProcessorAndThrowable(os, t, true);
msg = "{} " + msg + ": {}";
logger.debug(msg, os);
if (logger.isDebugEnabled()) {
logger.debug("", t);
}
logRepository.addLogMessage(LogLevel.DEBUG, msg, os, t);
}

Просмотреть файл

@ -178,6 +178,22 @@ The following binary components are provided under the Apache Software License v
Grok
Copyright 2014 Anthony Corbacho, and contributors.
(ASLv2) Apache Calcite
The following NOTICE information applies:
Apache Calcite
Copyright 2012-2017 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
This product is based on source code originally developed
by DynamoBI Corporation, LucidEra Inc., SQLstream Inc. and others
under the auspices of the Eigenbase Foundation
and released as the LucidDB project.
The web site includes files generated by Jekyll.
************************
Common Development and Distribution License 1.1
************************

Просмотреть файл

@ -1,15 +1,16 @@
<?xml version="1.0"?>
<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
license agreements. See the NOTICE file distributed with this work for additional
information regarding copyright ownership. The ASF licenses this file to
You under the Apache License, Version 2.0 (the "License"); you may not use
this file except in compliance with the License. You may obtain a copy of
the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
by applicable law or agreed to in writing, software distributed under the
License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
OF ANY KIND, either express or implied. See the License for the specific
language governing permissions and limitations under the License. -->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
license agreements. See the NOTICE file distributed with this work for additional
information regarding copyright ownership. The ASF licenses this file to
You under the Apache License, Version 2.0 (the "License"); you may not use
this file except in compliance with the License. You may obtain a copy of
the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
by applicable law or agreed to in writing, software distributed under the
License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
OF ANY KIND, either express or implied. See the License for the specific
language governing permissions and limitations under the License. -->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
@ -48,6 +49,10 @@ language governing permissions and limitations under the License. -->
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-http-context-map-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-service-api</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>

Просмотреть файл

@ -1,303 +0,0 @@
/*
* 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.calcite.adapter.csv;
import java.io.IOException;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.TimeZone;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.util.Pair;
import org.apache.commons.lang3.time.FastDateFormat;
import au.com.bytecode.opencsv.CSVReader;
/** Enumerator that reads from a CSV stream.
*
* @param <E> Row type
*/
class CsvEnumerator2<E> implements Enumerator<E> {
private final CSVReader reader;
private final String[] filterValues;
private final RowConverter<E> rowConverter;
private E current;
private static final FastDateFormat TIME_FORMAT_DATE;
private static final FastDateFormat TIME_FORMAT_TIME;
private static final FastDateFormat TIME_FORMAT_TIMESTAMP;
static {
TimeZone gmt = TimeZone.getTimeZone("GMT");
TIME_FORMAT_DATE = FastDateFormat.getInstance("yyyy-MM-dd", gmt);
TIME_FORMAT_TIME = FastDateFormat.getInstance("HH:mm:ss", gmt);
TIME_FORMAT_TIMESTAMP =
FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt);
}
public CsvEnumerator2(CSVReader csvReader, List<CsvFieldType> fieldTypes) {
this(verifyNotNullReader(csvReader), fieldTypes, identityList(fieldTypes.size()));
}
public CsvEnumerator2(CSVReader csvReader, List<CsvFieldType> fieldTypes, int[] fields) {
//noinspection unchecked
this(csvReader, null, (RowConverter<E>) converter(fieldTypes, fields));
}
public CsvEnumerator2(CSVReader csvReader, String[] filterValues, RowConverter<E> rowConverter) {
this.rowConverter = rowConverter;
this.filterValues = filterValues;
this.reader = csvReader;
}
static public CSVReader verifyNotNullReader(CSVReader csvReader) {
if (csvReader==null)
throw new IllegalArgumentException("csvReader cannot be null");
return csvReader;
}
private static RowConverter<?> converter(List<CsvFieldType> fieldTypes,
int[] fields) {
if (fields.length == 1) {
final int field = fields[0];
return new SingleColumnRowConverter(fieldTypes.get(field), field);
} else {
return new ArrayRowConverter(fieldTypes, fields);
}
}
/** Deduces the names and types of a table's columns by reading the first line
* of a CSV stream. */
static public RelDataType deduceRowType(JavaTypeFactory typeFactory, String[] firstLine,
List<CsvFieldType> fieldTypes) {
final List<RelDataType> types = new ArrayList<>();
final List<String> names = new ArrayList<>();
for (String string : firstLine) {
final String name;
final CsvFieldType fieldType;
final int colon = string.indexOf(':');
if (colon >= 0) {
name = string.substring(0, colon);
String typeString = string.substring(colon + 1);
typeString = typeString.trim();
fieldType = CsvFieldType.of(typeString);
if (fieldType == null) {
System.out.println("WARNING: Found unknown type: "
+ typeString + " in first line: "
+ " for column: " + name
+ ". Will assume the type of column is string");
}
} else {
name = string;
fieldType = null;
}
final RelDataType type;
if (fieldType == null) {
type = typeFactory.createJavaType(String.class);
} else {
type = fieldType.toType(typeFactory);
}
names.add(name);
types.add(type);
if (fieldTypes != null) {
fieldTypes.add(fieldType);
}
}
if (names.isEmpty()) {
names.add("line");
types.add(typeFactory.createJavaType(String.class));
}
return typeFactory.createStructType(Pair.zip(names, types));
}
public E current() {
return current;
}
public boolean moveNext() {
try {
outer:
for (;;) {
final String[] strings = reader.readNext();
if (strings == null) {
current = null;
reader.close();
return false;
}
if (filterValues != null) {
for (int i = 0; i < strings.length; i++) {
String filterValue = filterValues[i];
if (filterValue != null) {
if (!filterValue.equals(strings[i])) {
continue outer;
}
}
}
}
current = rowConverter.convertRow(strings);
return true;
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
public void reset() {
throw new UnsupportedOperationException();
}
public void close() {
try {
reader.close();
} catch (IOException e) {
throw new RuntimeException("Error closing CSV reader", e);
}
}
/** Returns an array of integers {0, ..., n - 1}. */
static int[] identityList(int n) {
int[] integers = new int[n];
for (int i = 0; i < n; i++) {
integers[i] = i;
}
return integers;
}
/** Row converter. */
abstract static class RowConverter<E> {
abstract E convertRow(String[] rows);
protected Object convert(CsvFieldType fieldType, String string) {
if (fieldType == null) {
return string;
}
switch (fieldType) {
case BOOLEAN:
if (string.length() == 0) {
return null;
}
return Boolean.parseBoolean(string);
case BYTE:
if (string.length() == 0) {
return null;
}
return Byte.parseByte(string);
case SHORT:
if (string.length() == 0) {
return null;
}
return Short.parseShort(string);
case INT:
if (string.length() == 0) {
return null;
}
return Integer.parseInt(string);
case LONG:
if (string.length() == 0) {
return null;
}
return Long.parseLong(string);
case FLOAT:
if (string.length() == 0) {
return null;
}
return Float.parseFloat(string);
case DOUBLE:
if (string.length() == 0) {
return null;
}
return Double.parseDouble(string);
case DATE:
if (string.length() == 0) {
return null;
}
try {
Date date = TIME_FORMAT_DATE.parse(string);
return new java.sql.Date(date.getTime());
} catch (ParseException e) {
return null;
}
case TIME:
if (string.length() == 0) {
return null;
}
try {
Date date = TIME_FORMAT_TIME.parse(string);
return new java.sql.Time(date.getTime());
} catch (ParseException e) {
return null;
}
case TIMESTAMP:
if (string.length() == 0) {
return null;
}
try {
Date date = TIME_FORMAT_TIMESTAMP.parse(string);
return new java.sql.Timestamp(date.getTime());
} catch (ParseException e) {
return null;
}
case STRING:
default:
return string;
}
}
}
/** Array row converter. */
static class ArrayRowConverter extends RowConverter<Object[]> {
private final CsvFieldType[] fieldTypes;
private final int[] fields;
ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields) {
this.fieldTypes = fieldTypes.toArray(new CsvFieldType[fieldTypes.size()]);
this.fields = fields;
}
public Object[] convertRow(String[] strings) {
final Object[] objects = new Object[fields.length];
for (int i = 0; i < fields.length; i++) {
int field = fields[i];
objects[i] = convert(fieldTypes[field], strings[field]);
}
return objects;
}
}
/** Single column row converter. */
private static class SingleColumnRowConverter extends RowConverter {
private final CsvFieldType fieldType;
private final int fieldIndex;
private SingleColumnRowConverter(CsvFieldType fieldType, int fieldIndex) {
this.fieldType = fieldType;
this.fieldIndex = fieldIndex;
}
public Object convertRow(String[] strings) {
return convert(fieldType, strings[fieldIndex]);
}
}
}
// End CsvEnumerator2.java

Просмотреть файл

@ -1,98 +0,0 @@
/*
* 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.calcite.adapter.csv;
import java.io.Reader;
import java.util.Map;
import org.apache.calcite.schema.Table;
import org.apache.calcite.schema.impl.AbstractSchema;
import com.google.common.collect.ImmutableMap;
/**
* Schema mapped onto a directory of CSV files. Each table in the schema
* is a CSV file in that directory.
*/
public class CsvSchema2 extends AbstractSchema {
final private Map<String, Reader> inputs;
private final CsvTable.Flavor flavor;
private Map<String, Table> tableMap;
/**
* Creates a CSV schema.
*
* @param inputs Inputs map
* @param flavor Whether to instantiate flavor tables that undergo
* query optimization
*/
public CsvSchema2(Map<String, Reader> inputs, CsvTable.Flavor flavor) {
super();
this.inputs = inputs;
this.flavor = flavor;
}
/** Looks for a suffix on a string and returns
* either the string with the suffix removed
* or the original string. */
private static String trim(String s, String suffix) {
String trimmed = trimOrNull(s, suffix);
return trimmed != null ? trimmed : s;
}
/** Looks for a suffix on a string and returns
* either the string with the suffix removed
* or null. */
private static String trimOrNull(String s, String suffix) {
return s.endsWith(suffix)
? s.substring(0, s.length() - suffix.length())
: null;
}
@Override protected Map<String, Table> getTableMap() {
if (tableMap!=null)
return tableMap;
// Build a map from table name to table; each file becomes a table.
final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
for (Map.Entry<String, Reader> entry : inputs.entrySet()) {
final Table table = createTable(entry.getValue());
builder.put(entry.getKey(), table);
}
tableMap = builder.build();
return tableMap;
}
/** Creates different sub-type of table based on the "flavor" attribute. */
private Table createTable(Reader readerx) {
switch (flavor) {
case TRANSLATABLE:
return new CsvTranslatableTable2(readerx, null);
// case SCANNABLE:
// return new CsvScannableTable(file, null);
// case FILTERABLE:
// return new CsvFilterableTable(file, null);
default:
throw new AssertionError("Unknown flavor " + flavor);
}
}
}
// End CsvSchema2.java

Просмотреть файл

@ -1,53 +0,0 @@
/*
* 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.calcite.adapter.csv;
import java.io.Reader;
import java.util.Map;
import org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.SchemaFactory;
import org.apache.calcite.schema.SchemaPlus;
/**
* Factory that creates a {@link CsvSchema}.
*
* <p>Allows a custom schema to be included in a <code><i>model</i>.json</code>
* file.</p>
*/
@SuppressWarnings("UnusedDeclaration")
public class CsvSchemaFactory2 implements SchemaFactory {
final private Map<String, Reader> inputs;
// public constructor, per factory contract
public CsvSchemaFactory2(Map<String, Reader> inputs) {
this.inputs = inputs;
}
public Schema create(SchemaPlus parentSchema, String name, Map<String, Object> operand) {
String flavorName = (String) operand.get("flavor");
CsvTable.Flavor flavor;
if (flavorName == null) {
flavor = CsvTable.Flavor.SCANNABLE;
} else {
flavor = CsvTable.Flavor.valueOf(flavorName.toUpperCase());
}
return new CsvSchema2(inputs, flavor);
}
}
// End CsvSchemaFactory2.java

Просмотреть файл

@ -1,104 +0,0 @@
/*
* 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.calcite.adapter.csv;
import org.apache.calcite.adapter.enumerable.EnumerableConvention;
import org.apache.calcite.adapter.enumerable.EnumerableRel;
import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
import org.apache.calcite.adapter.enumerable.PhysType;
import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
import org.apache.calcite.linq4j.tree.Blocks;
import org.apache.calcite.linq4j.tree.Expressions;
import org.apache.calcite.linq4j.tree.Primitive;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
import java.util.List;
/**
* Relational expression representing a scan of a CSV stream.
*
* <p>Like any table scan, it serves as a leaf node of a query tree.</p>
*/
public class CsvTableScan2 extends TableScan implements EnumerableRel {
final CsvTranslatableTable2 csvTable;
final int[] fields;
protected CsvTableScan2(RelOptCluster cluster, RelOptTable table,
CsvTranslatableTable2 csvTable, int[] fields) {
super(cluster, cluster.traitSetOf(EnumerableConvention.INSTANCE), table);
this.csvTable = csvTable;
this.fields = fields;
assert csvTable != null;
}
@Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
assert inputs.isEmpty();
return new CsvTableScan2(getCluster(), table, csvTable, fields);
}
@Override public RelWriter explainTerms(RelWriter pw) {
return super.explainTerms(pw)
.item("fields", Primitive.asList(fields));
}
@Override public RelDataType deriveRowType() {
final List<RelDataTypeField> fieldList = table.getRowType().getFieldList();
final RelDataTypeFactory.FieldInfoBuilder builder =
getCluster().getTypeFactory().builder();
for (int field : fields) {
builder.add(fieldList.get(field));
}
return builder.build();
}
@Override public void register(RelOptPlanner planner) {
planner.addRule(CsvProjectTableScanRule.INSTANCE);
}
public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
PhysType physType =
PhysTypeImpl.of(
implementor.getTypeFactory(),
getRowType(),
pref.preferArray());
if (table instanceof JsonTable) {
return implementor.result(
physType,
Blocks.toBlock(
Expressions.call(table.getExpression(JsonTable.class),
"enumerable")));
}
return implementor.result(
physType,
Blocks.toBlock(
Expressions.call(table.getExpression(CsvTranslatableTable2.class),
"project", Expressions.constant(fields))));
}
}
// End CsvTableScan.java

Просмотреть файл

@ -1,121 +0,0 @@
/*
* 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.calcite.adapter.csv;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.linq4j.AbstractEnumerable;
import org.apache.calcite.linq4j.Enumerable;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.calcite.linq4j.QueryProvider;
import org.apache.calcite.linq4j.Queryable;
import org.apache.calcite.linq4j.tree.Expression;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelProtoDataType;
import org.apache.calcite.schema.QueryableTable;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.schema.Schemas;
import org.apache.calcite.schema.TranslatableTable;
import au.com.bytecode.opencsv.CSVReader;
import java.io.IOException;
import java.io.Reader;
import java.lang.reflect.Type;
import java.util.ArrayList;
/**
* Table based on a CSV stream.
*/
public class CsvTranslatableTable2 extends CsvTable
implements QueryableTable, TranslatableTable {
final private CSVReader csvReader;
private CsvEnumerator2<Object> csvEnumerator2;
final private String[] firstLine;
/** Creates a CsvTable.
*/
CsvTranslatableTable2(Reader readerx, RelProtoDataType protoRowType) {
super(null, protoRowType);
this.csvReader = new CSVReader(readerx);
try {
this.firstLine = csvReader.readNext();
} catch (IOException e) {
throw new RuntimeException("csvReader.readNext() failed ", e);
}
}
public String toString() {
return "CsvTranslatableTable2";
}
/** Returns an enumerable over a given projection of the fields.
*
* <p>Called from generated code. */
public Enumerable<Object> project(final int[] fields) {
return new AbstractEnumerable<Object>() {
public Enumerator<Object> enumerator() {
return csvEnumerator2;
}
};
}
public Expression getExpression(SchemaPlus schema, String tableName,
Class clazz) {
return Schemas.tableExpression(schema, getElementType(), tableName, clazz);
}
public Type getElementType() {
return Object[].class;
}
public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
SchemaPlus schema, String tableName) {
throw new UnsupportedOperationException();
}
public RelNode toRel(
RelOptTable.ToRelContext context,
RelOptTable relOptTable) {
// Request all fields.
final int fieldCount = relOptTable.getRowType().getFieldCount();
final int[] fields = CsvEnumerator.identityList(fieldCount);
return new CsvTableScan2(context.getCluster(), relOptTable, this, fields);
}
@Override
public RelDataType getRowType(RelDataTypeFactory typeFactory) {
RelDataType rowType = null;
if (fieldTypes == null) {
fieldTypes = new ArrayList<CsvFieldType>();
rowType = CsvEnumerator2.deduceRowType((JavaTypeFactory) typeFactory, firstLine, fieldTypes);
} else {
rowType = CsvEnumerator2.deduceRowType((JavaTypeFactory) typeFactory, firstLine, null);
}
if (csvEnumerator2==null)
csvEnumerator2 = new CsvEnumerator2<Object>(csvReader, fieldTypes);
return rowType;
}
}
// End CsvTranslatableTable2.java

Просмотреть файл

@ -1,258 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.standard;
import static java.sql.Types.CHAR;
import static java.sql.Types.LONGNVARCHAR;
import static java.sql.Types.LONGVARCHAR;
import static java.sql.Types.NCHAR;
import static java.sql.Types.NVARCHAR;
import static java.sql.Types.VARCHAR;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.Reader;
import java.nio.charset.StandardCharsets;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.calcite.adapter.csv.CsvSchemaFactory2;
import org.apache.calcite.jdbc.CalciteConnection;
import org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.commons.lang3.StringEscapeUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ProcessorLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.StreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.util.StopWatch;
import com.google.common.collect.ImmutableMap;
@EventDriven
@SideEffectFree
@SupportsBatching
@Tags({"xml", "xslt", "transform"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Filter out specific columns from CSV data. Some other transformations are also supported."
+ "Columns can be renamed, simple calculations performed, aggregations, etc."
+ "SQL select statement is used to specify how CSV data should be transformed."
+ "SQL statement follows standard SQL, some restrictions may apply."
+ "Successfully transformed CSV data is routed to the 'success' relationship."
+ "If transform fails, the original FlowFile is routed to the 'failure' relationship")
public class FilterCSVColumns extends AbstractProcessor {
public static final PropertyDescriptor SQL_SELECT = new PropertyDescriptor.Builder()
.name("SQL select statement")
.description("SQL select statement specifies how CSV data should be transformed. "
+ "Sql select should select from CSV.A table")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("The FlowFile with transformed content will be routed to this relationship")
.build();
public static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description("If a FlowFile fails processing for any reason (for example, the SQL statement contains columns not present in CSV), it will be routed to this relationship")
.build();
private List<PropertyDescriptor> properties;
private Set<Relationship> relationships;
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(SQL_SELECT);
this.properties = Collections.unmodifiableList(properties);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
relationships.add(REL_FAILURE);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
final FlowFile original = session.get();
if (original == null) {
return;
}
final ProcessorLog logger = getLogger();
final StopWatch stopWatch = new StopWatch(true);
try {
FlowFile transformed = session.write(original, new StreamCallback() {
@Override
public void process(final InputStream rawIn, final OutputStream out) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) {
String sql = context.getProperty(SQL_SELECT).getValue();
final ResultSet resultSet = transform(rawIn, sql);
convertToCSV(resultSet, out);
} catch (final Exception e) {
throw new IOException(e);
}
}
});
session.transfer(transformed, REL_SUCCESS);
session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
logger.info("Transformed {}", new Object[]{original});
} catch (ProcessException e) {
logger.error("Unable to transform {} due to {}", new Object[]{original, e});
session.transfer(original, REL_FAILURE);
}
}
static protected ResultSet transform(InputStream rawIn, String sql) throws SQLException {
Reader readerx = new InputStreamReader(rawIn);
HashMap<String, Reader> inputs = new HashMap<>();
inputs.put("A", readerx);
Statement statement = null;
final Properties properties = new Properties();
// properties.setProperty("caseSensitive", "true");
try (final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties)) {
final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
final SchemaPlus rootSchema = calciteConnection.getRootSchema();
final Schema schema =
new CsvSchemaFactory2(inputs)
.create(rootSchema, "CSV", ImmutableMap.<String, Object>of("flavor", "TRANSLATABLE"));
calciteConnection.getRootSchema().add("CSV", schema);
rootSchema.add("default", schema);
statement = connection.createStatement();
final ResultSet resultSet = statement.executeQuery(sql);
return resultSet;
}
}
static protected void convertToCSV(ResultSet resultSet, OutputStream out) throws SQLException, IOException {
convertToCsvStream(resultSet, out);
}
public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream) throws SQLException, IOException {
return convertToCsvStream(rs, outStream, null, null);
}
public static long convertToCsvStream(final ResultSet rs, final OutputStream outStream, String recordName, ResultSetRowCallback callback)
throws SQLException, IOException {
final ResultSetMetaData meta = rs.getMetaData();
final int nrOfColumns = meta.getColumnCount();
List<String> columnNames = new ArrayList<>(nrOfColumns);
for (int i = 1; i <= nrOfColumns; i++) {
String columnNameFromMeta = meta.getColumnName(i);
// Hive returns table.column for column name. Grab the column name as the string after the last period
int columnNameDelimiter = columnNameFromMeta.lastIndexOf(".");
columnNames.add(columnNameFromMeta.substring(columnNameDelimiter + 1));
}
// Write column names as header row
outStream.write(StringUtils.join(columnNames, ",").getBytes(StandardCharsets.UTF_8));
outStream.write("\n".getBytes(StandardCharsets.UTF_8));
// Iterate over the rows
long nrOfRows = 0;
while (rs.next()) {
if (callback != null) {
callback.processRow(rs);
}
List<String> rowValues = new ArrayList<>(nrOfColumns);
for (int i = 1; i <= nrOfColumns; i++) {
final int javaSqlType = meta.getColumnType(i);
final Object value = rs.getObject(i);
switch (javaSqlType) {
case CHAR:
case LONGNVARCHAR:
case LONGVARCHAR:
case NCHAR:
case NVARCHAR:
case VARCHAR:
rowValues.add("\"" + StringEscapeUtils.escapeCsv(rs.getString(i)) + "\"");
break;
default:
rowValues.add(value.toString());
}
}
// Write row values
outStream.write(StringUtils.join(rowValues, ",").getBytes(StandardCharsets.UTF_8));
outStream.write("\n".getBytes(StandardCharsets.UTF_8));
nrOfRows++;
}
return nrOfRows;
}
/**
* An interface for callback methods which allows processing of a row during the convertToXYZStream() processing.
* <b>IMPORTANT:</b> This method should only work on the row pointed at by the current ResultSet reference.
* Advancing the cursor (e.g.) can cause rows to be skipped during Avro transformation.
*/
public interface ResultSetRowCallback {
void processRow(ResultSet resultSet) throws IOException;
}
}

Просмотреть файл

@ -0,0 +1,541 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.standard;
import java.io.Closeable;
import java.io.IOException;
import java.io.OutputStream;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import org.apache.calcite.config.CalciteConnectionProperty;
import org.apache.calcite.config.Lex;
import org.apache.calcite.jdbc.CalciteConnection;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.sql.parser.SqlParser;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.DynamicRelationship;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.queryflowfile.FlowFileTable;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.ResultSetRecordSet;
import org.apache.nifi.util.StopWatch;
@EventDriven
@SideEffectFree
@SupportsBatching
@Tags({"sql", "query", "calcite", "route", "record", "transform", "select", "update", "modify", "etl", "filter", "record", "csv", "json", "logs", "text", "avro", "aggregate"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Evaluates one or more SQL queries against the contents of a FlowFile. The result of the "
+ "SQL query then becomes the content of the output FlowFile. This can be used, for example, "
+ "for field-specific filtering, transformation, and row-level filtering. "
+ "Columns can be renamed, simple calculations and aggregations performed, etc. "
+ "The Processor is configured with a Record Reader Controller Service and a Record Writer service so as to allow flexibility in incoming and outgoing data formats. "
+ "The Processor must be configured with at least one user-defined property. The name of the Property "
+ "is the Relationship to route data to, and the value of the Property is a SQL SELECT statement that is used to specify how input data should be transformed/filtered. "
+ "The SQL statement must be valid ANSI SQL and is powered by Apache Calcite. "
+ "If the transformation fails, the original FlowFile is routed to the 'failure' relationship. Otherwise, the data selected will be routed to the associated "
+ "relationship. See the Processor Usage documentation for more information.")
@DynamicRelationship(name="<Property Name>", description="Each user-defined property defines a new Relationship for this Processor.")
@DynamicProperty(name = "The name of the relationship to route data to", value="A SQL SELECT statement that is used to determine what data should be routed to this "
+ "relationship.", supportsExpressionLanguage=true, description="Each user-defined property specifies a SQL SELECT statement to run over the data, with the data "
+ "that is selected being routed to the relationship whose name is the property name")
public class QueryFlowFile extends AbstractProcessor {
static final PropertyDescriptor RECORD_READER_FACTORY = new PropertyDescriptor.Builder()
.name("Record Reader")
.description("Specifies the Controller Service to use for parsing incoming data and determining the data's schema")
.identifiesControllerService(RowRecordReaderFactory.class)
.required(true)
.build();
static final PropertyDescriptor RECORD_WRITER_FACTORY = new PropertyDescriptor.Builder()
.name("Record Writer")
.description("Specifies the Controller Service to use for writing results to a FlowFile")
.identifiesControllerService(RecordSetWriterFactory.class)
.required(true)
.build();
static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
.name("Include Zero Record FlowFiles")
.description("When running the SQL statement against an incoming FlowFile, if the result has no data, "
+ "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
.expressionLanguageSupported(false)
.allowableValues("true", "false")
.defaultValue("true")
.required(true)
.build();
static final PropertyDescriptor CACHE_SCHEMA = new PropertyDescriptor.Builder()
.name("Cache Schema")
.description("Parsing the SQL query and deriving the FlowFile's schema is relatively expensive. If this value is set to true, "
+ "the Processor will cache these values so that the Processor is much more efficient and much faster. However, if this is done, "
+ "then the schema that is derived for the first FlowFile processed must apply to all FlowFiles. If all FlowFiles will not have the exact "
+ "same schema, or if the SQL SELECT statement uses the Expression Language, this value should be set to false.")
.expressionLanguageSupported(false)
.allowableValues("true", "false")
.defaultValue("true")
.required(true)
.build();
public static final Relationship REL_ORIGINAL = new Relationship.Builder()
.name("original")
.description("The original FlowFile is routed to this relationship")
.build();
public static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description("If a FlowFile fails processing for any reason (for example, the SQL "
+ "statement contains columns not present in input data), the original FlowFile it will "
+ "be routed to this relationship")
.build();
private List<PropertyDescriptor> properties;
private final Set<Relationship> relationships = Collections.synchronizedSet(new HashSet<>());
private final Map<String, BlockingQueue<CachedStatement>> statementQueues = new HashMap<>();
@Override
protected void init(final ProcessorInitializationContext context) {
try {
DriverManager.registerDriver(new org.apache.calcite.jdbc.Driver());
} catch (final SQLException e) {
throw new ProcessException("Failed to load Calcite JDBC Driver", e);
}
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(RECORD_READER_FACTORY);
properties.add(RECORD_WRITER_FACTORY);
properties.add(INCLUDE_ZERO_RECORD_FLOWFILES);
properties.add(CACHE_SCHEMA);
this.properties = Collections.unmodifiableList(properties);
relationships.add(REL_FAILURE);
relationships.add(REL_ORIGINAL);
}
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
@Override
public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
if (!descriptor.isDynamic()) {
return;
}
final Relationship relationship = new Relationship.Builder()
.name(descriptor.getName())
.description("User-defined relationship that specifies where data that matches the specified SQL query should be routed")
.build();
if (newValue == null) {
relationships.remove(relationship);
} else {
relationships.add(relationship);
}
}
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final boolean cache = validationContext.getProperty(CACHE_SCHEMA).asBoolean();
if (cache) {
for (final PropertyDescriptor descriptor : validationContext.getProperties().keySet()) {
if (descriptor.isDynamic() && validationContext.isExpressionLanguagePresent(validationContext.getProperty(descriptor).getValue())) {
return Collections.singleton(new ValidationResult.Builder()
.subject("Cache Schema")
.input("true")
.valid(false)
.explanation("Cannot have 'Cache Schema' property set to true if any SQL statement makes use of the Expression Language")
.build());
}
}
}
return Collections.emptyList();
}
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.description("SQL select statement specifies how data should be filtered/transformed. "
+ "SQL SELECT should select from the FLOWFILE table")
.required(false)
.dynamic(true)
.expressionLanguageSupported(true)
.addValidator(new SqlValidator())
.build();
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
final FlowFile original = session.get();
if (original == null) {
return;
}
final StopWatch stopWatch = new StopWatch(true);
final RecordSetWriterFactory resultSetWriterFactory = context.getProperty(RECORD_WRITER_FACTORY)
.asControllerService(RecordSetWriterFactory.class);
final RowRecordReaderFactory recordParserFactory = context.getProperty(RECORD_READER_FACTORY)
.asControllerService(RowRecordReaderFactory.class);
final RecordSetWriter resultSetWriter = resultSetWriterFactory.createWriter(getLogger());
final Map<FlowFile, Relationship> transformedFlowFiles = new HashMap<>();
final Set<FlowFile> createdFlowFiles = new HashSet<>();
try {
for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
if (!descriptor.isDynamic()) {
continue;
}
final Relationship relationship = new Relationship.Builder().name(descriptor.getName()).build();
// We have to fork a child because we may need to read the input FlowFile more than once,
// and we cannot call session.read() on the original FlowFile while we are within a write
// callback for the original FlowFile.
FlowFile transformed = session.create(original);
// Ensure that we have the FlowFile in the map in case we throw any Exception
createdFlowFiles.add(transformed);
final String sql = context.getProperty(descriptor).evaluateAttributeExpressions(original).getValue();
final AtomicReference<WriteResult> writeResultRef = new AtomicReference<>();
final QueryResult queryResult;
if (context.getProperty(CACHE_SCHEMA).asBoolean()) {
queryResult = queryWithCache(session, original, sql, context, recordParserFactory);
} else {
queryResult = query(session, original, sql, context, recordParserFactory);
}
try {
final ResultSet rs = queryResult.getResultSet();
transformed = session.write(transformed, new OutputStreamCallback() {
@Override
public void process(final OutputStream out) throws IOException {
try {
final ResultSetRecordSet recordSet = new ResultSetRecordSet(rs);
writeResultRef.set(resultSetWriter.write(recordSet, out));
} catch (final Exception e) {
throw new IOException(e);
}
}
});
} finally {
closeQuietly(queryResult);
}
final WriteResult result = writeResultRef.get();
if (result.getRecordCount() == 0 && !context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean()) {
session.remove(transformed);
transformedFlowFiles.remove(transformed);
getLogger().info("Transformed {} but the result contained no data so will not pass on a FlowFile", new Object[] {original});
} else {
final Map<String, String> attributesToAdd = new HashMap<>();
if (result.getAttributes() != null) {
attributesToAdd.putAll(result.getAttributes());
}
attributesToAdd.put(CoreAttributes.MIME_TYPE.key(), resultSetWriter.getMimeType());
attributesToAdd.put("record.count", String.valueOf(result.getRecordCount()));
transformed = session.putAllAttributes(transformed, attributesToAdd);
transformedFlowFiles.put(transformed, relationship);
}
}
final long elapsedMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS);
if (transformedFlowFiles.size() > 0) {
session.getProvenanceReporter().fork(original, transformedFlowFiles.keySet(), elapsedMillis);
for (final Map.Entry<FlowFile, Relationship> entry : transformedFlowFiles.entrySet()) {
final FlowFile transformed = entry.getKey();
final Relationship relationship = entry.getValue();
session.getProvenanceReporter().route(transformed, relationship);
session.transfer(transformed, relationship);
}
}
getLogger().info("Successfully transformed {} in {} millis", new Object[] {original, elapsedMillis});
session.transfer(original, REL_ORIGINAL);
} catch (ProcessException e) {
getLogger().error("Unable to transform {} due to {}", new Object[] {original, e});
session.remove(createdFlowFiles);
session.transfer(original, REL_FAILURE);
} catch (final SQLException e) {
getLogger().error("Unable to transform {} due to {}", new Object[] {original, e.getCause() == null ? e : e.getCause()});
session.remove(createdFlowFiles);
session.transfer(original, REL_FAILURE);
}
}
private synchronized CachedStatement getStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
final FlowFile flowFile, final RowRecordReaderFactory recordReaderFactory) throws SQLException {
final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
if (statementQueue == null) {
return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
}
final CachedStatement cachedStmt = statementQueue.poll();
if (cachedStmt != null) {
return cachedStmt;
}
return buildCachedStatement(sql, connectionSupplier, session, flowFile, recordReaderFactory);
}
private CachedStatement buildCachedStatement(final String sql, final Supplier<CalciteConnection> connectionSupplier, final ProcessSession session,
final FlowFile flowFile, final RowRecordReaderFactory recordReaderFactory) throws SQLException {
final CalciteConnection connection = connectionSupplier.get();
final SchemaPlus rootSchema = connection.getRootSchema();
final FlowFileTable<?, ?> flowFileTable = new FlowFileTable<>(session, flowFile, recordReaderFactory, getLogger());
rootSchema.add("FLOWFILE", flowFileTable);
rootSchema.setCacheEnabled(false);
final PreparedStatement stmt = connection.prepareStatement(sql);
return new CachedStatement(stmt, flowFileTable, connection);
}
@OnStopped
public synchronized void cleanup() {
for (final BlockingQueue<CachedStatement> statementQueue : statementQueues.values()) {
CachedStatement stmt;
while ((stmt = statementQueue.poll()) != null) {
closeQuietly(stmt.getStatement(), stmt.getConnection());
}
}
statementQueues.clear();
}
@OnScheduled
public synchronized void setupQueues(final ProcessContext context) {
// Create a Queue of PreparedStatements for each property that is user-defined. This allows us to easily poll the
// queue and add as necessary, knowing that the queue already exists.
for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
if (!descriptor.isDynamic()) {
continue;
}
final String sql = context.getProperty(descriptor).evaluateAttributeExpressions().getValue();
final BlockingQueue<CachedStatement> queue = new LinkedBlockingQueue<>(context.getMaxConcurrentTasks());
statementQueues.put(sql, queue);
}
}
protected QueryResult queryWithCache(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context,
final RowRecordReaderFactory recordParserFactory) throws SQLException {
final Supplier<CalciteConnection> connectionSupplier = () -> {
final Properties properties = new Properties();
properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.JAVA.name());
try {
final Connection connection = DriverManager.getConnection("jdbc:calcite:", properties);
final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
return calciteConnection;
} catch (final Exception e) {
throw new ProcessException(e);
}
};
final CachedStatement cachedStatement = getStatement(sql, connectionSupplier, session, flowFile, recordParserFactory);
final PreparedStatement stmt = cachedStatement.getStatement();
final FlowFileTable<?, ?> table = cachedStatement.getTable();
table.setFlowFile(session, flowFile);
final ResultSet rs = stmt.executeQuery();
return new QueryResult() {
@Override
public void close() throws IOException {
final BlockingQueue<CachedStatement> statementQueue = statementQueues.get(sql);
if (statementQueue == null || !statementQueue.offer(cachedStatement)) {
try {
cachedStatement.getConnection().close();
} catch (SQLException e) {
throw new IOException("Failed to close statement", e);
}
}
}
@Override
public ResultSet getResultSet() {
return rs;
}
};
}
protected QueryResult query(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context,
final RowRecordReaderFactory recordParserFactory) throws SQLException {
final Properties properties = new Properties();
properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.JAVA.name());
Connection connection = null;
ResultSet resultSet = null;
Statement statement = null;
try {
connection = DriverManager.getConnection("jdbc:calcite:", properties);
final CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
final SchemaPlus rootSchema = calciteConnection.getRootSchema();
final FlowFileTable<?, ?> flowFileTable = new FlowFileTable<>(session, flowFile, recordParserFactory, getLogger());
rootSchema.add("FLOWFILE", flowFileTable);
rootSchema.setCacheEnabled(false);
statement = connection.createStatement();
resultSet = statement.executeQuery(sql);
final ResultSet rs = resultSet;
final Statement stmt = statement;
final Connection conn = connection;
return new QueryResult() {
@Override
public void close() throws IOException {
closeQuietly(rs, stmt, conn);
}
@Override
public ResultSet getResultSet() {
return rs;
}
};
} catch (final Exception e) {
closeQuietly(resultSet, statement, connection);
throw e;
}
}
private void closeQuietly(final AutoCloseable... closeables) {
if (closeables == null) {
return;
}
for (final AutoCloseable closeable : closeables) {
if (closeable == null) {
continue;
}
try {
closeable.close();
} catch (final Exception e) {
getLogger().warn("Failed to close SQL resource", e);
}
}
}
private static class SqlValidator implements Validator {
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
final String substituted = context.newPropertyValue(input).evaluateAttributeExpressions().getValue();
final SqlParser parser = SqlParser.create(substituted);
try {
parser.parseStmt();
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(true)
.build();
} catch (final Exception e) {
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(false)
.explanation("Not a valid SQL Statement: " + e.getMessage())
.build();
}
}
}
private static interface QueryResult extends Closeable {
ResultSet getResultSet();
}
private static class CachedStatement {
private final FlowFileTable<?, ?> table;
private final PreparedStatement statement;
private final Connection connection;
public CachedStatement(final PreparedStatement statement, final FlowFileTable<?, ?> table, final Connection connection) {
this.statement = statement;
this.table = table;
this.connection = connection;
}
public FlowFileTable<?, ?> getTable() {
return table;
}
public PreparedStatement getStatement() {
return statement;
}
public Connection getConnection() {
return connection;
}
}
}

Просмотреть файл

@ -0,0 +1,150 @@
/*
* 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.queryflowfile;
import java.io.IOException;
import java.io.InputStream;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.record.Record;
public class FlowFileEnumerator<InternalType> implements Enumerator<Object> {
private final ProcessSession session;
private final FlowFile flowFile;
private final ComponentLog logger;
private final RowRecordReaderFactory recordParserFactory;
private final int[] fields;
private InputStream rawIn;
private Object currentRow;
private RecordReader recordParser;
public FlowFileEnumerator(final ProcessSession session, final FlowFile flowFile, final ComponentLog logger, final RowRecordReaderFactory parserFactory, final int[] fields) {
this.session = session;
this.flowFile = flowFile;
this.recordParserFactory = parserFactory;
this.logger = logger;
this.fields = fields;
reset();
}
@Override
public Object current() {
return currentRow;
}
@Override
public boolean moveNext() {
currentRow = null;
while (currentRow == null) {
try {
currentRow = filterColumns(recordParser.nextRecord());
break;
} catch (final IOException e) {
logger.error("Failed to read next record in stream for " + flowFile + ". Assuming end of stream.", e);
currentRow = null;
break;
} catch (final MalformedRecordException mre) {
logger.error("Failed to parse record in stream for " + flowFile + ". Will skip record and continue reading", mre);
}
}
if (currentRow == null) {
// If we are out of data, close the InputStream. We do this because
// Calcite does not necessarily call our close() method.
close();
}
return (currentRow != null);
}
private Object filterColumns(final Record record) {
if (record == null) {
return null;
}
final Object[] row = record.getValues();
// If we want no fields or if the row is null, just return null
if (fields == null || row == null) {
return row;
}
// If we want only a single field, then Calcite is going to expect us to return
// the actual value, NOT a 1-element array of values.
if (fields.length == 1) {
final int desiredCellIndex = fields[0];
return row[desiredCellIndex];
}
// Create a new Object array that contains only the desired fields.
if (row.length <= fields.length) {
return row;
}
final Object[] filtered = new Object[fields.length];
for (int i = 0; i < fields.length; i++) {
final int indexToKeep = fields[i];
filtered[i] = row[indexToKeep];
}
return filtered;
}
@Override
public void reset() {
if (rawIn != null) {
try {
rawIn.close();
} catch (final Exception e) {
logger.warn("Could not close FlowFile's input due to " + e, e);
}
}
rawIn = session.read(flowFile);
try {
recordParser = recordParserFactory.createRecordReader(rawIn, logger);
} catch (final MalformedRecordException | IOException e) {
throw new ProcessException("Failed to reset stream", e);
}
}
@Override
public void close() {
if (recordParser != null) {
try {
recordParser.close();
} catch (final Exception e) {
logger.warn("Failed to close decorated source for " + flowFile, e);
}
}
try {
rawIn.close();
} catch (final Exception e) {
logger.warn("Failed to close InputStream for " + flowFile, e);
}
}
}

Просмотреть файл

@ -0,0 +1,76 @@
/*
* 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.queryflowfile;
import java.util.List;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.logical.LogicalProject;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexNode;
/**
* Planner rule that projects from a {@link FlowFileTableScan} scan just the columns
* needed to satisfy a projection. If the projection's expressions are trivial,
* the projection is removed.
*/
public class FlowFileProjectTableScanRule extends RelOptRule {
public static final FlowFileProjectTableScanRule INSTANCE = new FlowFileProjectTableScanRule();
private FlowFileProjectTableScanRule() {
super(
operand(LogicalProject.class,
operand(FlowFileTableScan.class, none())),
"FlowFileProjectTableScanRule");
}
@Override
public void onMatch(RelOptRuleCall call) {
final LogicalProject project = call.rel(0);
final FlowFileTableScan scan = call.rel(1);
final int[] fields = getProjectFields(project.getProjects());
if (fields == null) {
// Project contains expressions more complex than just field references.
return;
}
call.transformTo(
new FlowFileTableScan(
scan.getCluster(),
scan.getTable(),
scan.flowFileTable,
fields));
}
private int[] getProjectFields(List<RexNode> exps) {
final int[] fields = new int[exps.size()];
for (int i = 0; i < exps.size(); i++) {
final RexNode exp = exps.get(i);
if (exp instanceof RexInputRef) {
fields[i] = ((RexInputRef) exp).getIndex();
} else {
return null; // not a simple projection
}
}
return fields;
}
}

Просмотреть файл

@ -0,0 +1,203 @@
/*
* 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.queryflowfile;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Type;
import java.util.ArrayList;
import java.util.List;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.linq4j.AbstractEnumerable;
import org.apache.calcite.linq4j.Enumerable;
import org.apache.calcite.linq4j.Enumerator;
import org.apache.calcite.linq4j.QueryProvider;
import org.apache.calcite.linq4j.Queryable;
import org.apache.calcite.linq4j.tree.Expression;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.schema.QueryableTable;
import org.apache.calcite.schema.Schema.TableType;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.schema.Schemas;
import org.apache.calcite.schema.TranslatableTable;
import org.apache.calcite.schema.impl.AbstractTable;
import org.apache.calcite.util.Pair;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.RecordField;
import org.apache.nifi.serialization.record.RecordSchema;
public class FlowFileTable<S, E> extends AbstractTable implements QueryableTable, TranslatableTable {
private final RowRecordReaderFactory recordParserFactory;
private final ComponentLog logger;
private RecordSchema recordSchema;
private RelDataType relDataType = null;
private volatile ProcessSession session;
private volatile FlowFile flowFile;
/**
* Creates a FlowFile table.
*/
public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RowRecordReaderFactory recordParserFactory, final ComponentLog logger) {
this.session = session;
this.flowFile = flowFile;
this.recordParserFactory = recordParserFactory;
this.logger = logger;
}
public void setFlowFile(final ProcessSession session, final FlowFile flowFile) {
this.session = session;
this.flowFile = flowFile;
}
@Override
public String toString() {
return "FlowFileTable";
}
/**
* Returns an enumerable over a given projection of the fields.
*
* <p>
* Called from generated code.
*/
public Enumerable<Object> project(final int[] fields) {
return new AbstractEnumerable<Object>() {
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public Enumerator<Object> enumerator() {
return new FlowFileEnumerator(session, flowFile, logger, recordParserFactory, fields);
}
};
}
@Override
@SuppressWarnings("rawtypes")
public Expression getExpression(final SchemaPlus schema, final String tableName, final Class clazz) {
return Schemas.tableExpression(schema, getElementType(), tableName, clazz);
}
@Override
public Type getElementType() {
return Object[].class;
}
@Override
public <T> Queryable<T> asQueryable(final QueryProvider queryProvider, final SchemaPlus schema, final String tableName) {
throw new UnsupportedOperationException();
}
@Override
public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable relOptTable) {
// Request all fields.
final int fieldCount = relOptTable.getRowType().getFieldCount();
final int[] fields = new int[fieldCount];
for (int i = 0; i < fieldCount; i++) {
fields[i] = i;
}
return new FlowFileTableScan(context.getCluster(), relOptTable, this, fields);
}
@Override
public RelDataType getRowType(final RelDataTypeFactory typeFactory) {
if (relDataType != null) {
return relDataType;
}
RecordSchema schema;
try (final InputStream in = session.read(flowFile)) {
final RecordReader recordParser = recordParserFactory.createRecordReader(in, logger);
schema = recordParser.getSchema();
} catch (final MalformedRecordException | IOException e) {
throw new ProcessException("Failed to determine schema of data records for " + flowFile, e);
}
final List<String> names = new ArrayList<>();
final List<RelDataType> types = new ArrayList<>();
final JavaTypeFactory javaTypeFactory = (JavaTypeFactory) typeFactory;
for (final RecordField field : schema.getFields()) {
names.add(field.getFieldName());
types.add(getRelDataType(field.getDataType(), javaTypeFactory));
}
logger.debug("Found Schema: {}", new Object[] {schema});
if (recordSchema == null) {
recordSchema = schema;
}
relDataType = typeFactory.createStructType(Pair.zip(names, types));
return relDataType;
}
private RelDataType getRelDataType(final DataType fieldType, final JavaTypeFactory typeFactory) {
switch (fieldType.getFieldType()) {
case BOOLEAN:
return typeFactory.createJavaType(boolean.class);
case BYTE:
return typeFactory.createJavaType(byte.class);
case CHAR:
return typeFactory.createJavaType(char.class);
case DATE:
return typeFactory.createJavaType(java.sql.Date.class);
case DOUBLE:
return typeFactory.createJavaType(double.class);
case FLOAT:
return typeFactory.createJavaType(float.class);
case INT:
return typeFactory.createJavaType(int.class);
case SHORT:
return typeFactory.createJavaType(short.class);
case TIME:
return typeFactory.createJavaType(java.sql.Time.class);
case TIMESTAMP:
return typeFactory.createJavaType(java.sql.Timestamp.class);
case LONG:
return typeFactory.createJavaType(long.class);
case STRING:
return typeFactory.createJavaType(String.class);
case ARRAY:
return typeFactory.createJavaType(Object[].class);
case OBJECT:
return typeFactory.createJavaType(Object.class);
}
throw new IllegalArgumentException("Unknown Record Field Type: " + fieldType);
}
@Override
public TableType getJdbcTableType() {
return TableType.TEMPORARY_TABLE;
}
}

Просмотреть файл

@ -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.queryflowfile;
import java.util.List;
import org.apache.calcite.adapter.enumerable.EnumerableConvention;
import org.apache.calcite.adapter.enumerable.EnumerableRel;
import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
import org.apache.calcite.adapter.enumerable.PhysType;
import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
import org.apache.calcite.linq4j.tree.Blocks;
import org.apache.calcite.linq4j.tree.Expressions;
import org.apache.calcite.linq4j.tree.Primitive;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelOptPlanner;
import org.apache.calcite.plan.RelOptTable;
import org.apache.calcite.plan.RelTraitSet;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelWriter;
import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
/**
* Relational expression representing a scan of a FlowFile.
*
* <p>
* Like any table scan, it serves as a leaf node of a query tree.
* </p>
*/
public class FlowFileTableScan extends TableScan implements EnumerableRel {
final FlowFileTable<?, ?> flowFileTable;
final int[] fields;
protected FlowFileTableScan(final RelOptCluster cluster, final RelOptTable table, final FlowFileTable<?, ?> flowFileTable, final int[] fields) {
super(cluster, cluster.traitSetOf(EnumerableConvention.INSTANCE), table);
this.flowFileTable = flowFileTable;
this.fields = fields;
}
@Override
public RelNode copy(final RelTraitSet traitSet, final List<RelNode> inputs) {
return new FlowFileTableScan(getCluster(), table, flowFileTable, fields);
}
@Override
public RelWriter explainTerms(final RelWriter pw) {
return super.explainTerms(pw).item("fields", Primitive.asList(fields));
}
@Override
public RelDataType deriveRowType() {
final List<RelDataTypeField> fieldList = table.getRowType().getFieldList();
final RelDataTypeFactory.FieldInfoBuilder builder = getCluster().getTypeFactory().builder();
for (int field : fields) {
builder.add(fieldList.get(field));
}
return builder.build();
}
@Override
public void register(RelOptPlanner planner) {
planner.addRule(FlowFileProjectTableScanRule.INSTANCE);
}
@Override
public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
return implementor.result(physType, Blocks.toBlock(
Expressions.call(table.getExpression(FlowFileTable.class), "project", Expressions.constant(fields))));
}
}

Просмотреть файл

@ -75,6 +75,7 @@ org.apache.nifi.processors.standard.PutSyslog
org.apache.nifi.processors.standard.PutTCP
org.apache.nifi.processors.standard.PutUDP
org.apache.nifi.processors.standard.QueryDatabaseTable
org.apache.nifi.processors.standard.QueryFlowFile
org.apache.nifi.processors.standard.ReplaceText
org.apache.nifi.processors.standard.RouteText
org.apache.nifi.processors.standard.ReplaceTextWithMapping

Просмотреть файл

@ -0,0 +1,47 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>QueryFlowFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>
QueryFlowFile provides users a tremendous amount of power by leveraging an extremely well-known
syntax (SQL) to route, filter, transform, and query data as it traverses the system. In order to
provide the Processor with the maximum amount of flexibility, it is configured with a Controller
Service that is responsible for reading and parsing the incoming FlowFiles and a Controller Service
that is responsible for writing the results out. By using this paradigm, users are not forced to
convert their data from one format to another just to query it, and then transform the data back
into the form that they want. Rather, the appropriate Controller Service can easily be configured
and put to use for the appropriate data format.
</p>
<p>
Rather than providing a single "SQL SELECT Statement" type of Property, this Processor makes use
of user-defined properties. Each user-defined property that is added to the Processor has a name
that becomes a new Relationship for the Processor and a corresponding SQL query that will be evaluated
against each FlowFile. This allows multiple SQL queries to be run against each FlowFile.
</p>
<p>
The SQL syntax that is supported by this Processor is ANSI SQL and is powered by Apache Calcite.
</p>
</body>
</html>

Просмотреть файл

@ -1,117 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.standard;
import static org.junit.Assert.assertEquals;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.List;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestFilterCSVColumns {
private static final Logger LOGGER;
static {
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.FilterCSVColumns", "debug");
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestFilterCSVColumns", "debug");
LOGGER = LoggerFactory.getLogger(TestFilterCSVColumns.class);
}
@Test
public void testTransformSimple() throws InitializationException, IOException, SQLException {
String sql = "select first_name, last_name, company_name, address, city from CSV.A where city='New York'";
Path inpath = Paths.get("src/test/resources/TestFilterCSVColumns/US500.csv");
InputStream in = new FileInputStream(inpath.toFile());
ResultSet resultSet = FilterCSVColumns.transform(in, sql);
int nrofColumns = resultSet.getMetaData().getColumnCount();
for (int i = 1; i <= nrofColumns; i++) {
System.out.print(resultSet.getMetaData().getColumnLabel(i) + " ");
}
System.out.println();
while (resultSet.next()) {
for (int i = 1; i <= nrofColumns; i++) {
System.out.print(resultSet.getString(i)+ " ");
}
System.out.println();
}
}
@Test
public void testTransformCalc() throws InitializationException, IOException, SQLException {
String sql = "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from CSV.A where ID=100";
Path inpath = Paths.get("src/test/resources/TestFilterCSVColumns/Numeric.csv");
InputStream in = new FileInputStream(inpath.toFile());
ResultSet resultSet = FilterCSVColumns.transform(in, sql);
int nrofColumns = resultSet.getMetaData().getColumnCount();
for (int i = 1; i <= nrofColumns; i++) {
System.out.print(resultSet.getMetaData().getColumnLabel(i) + " ");
}
System.out.println();
while (resultSet.next()) {
for (int i = 1; i <= nrofColumns; i++) {
System.out.print(resultSet.getString(i)+ " ");
}
double total = resultSet.getDouble("TOTAL");
System.out.println();
assertEquals(90.75, total, 0.0001);
}
}
@Test
public void testSimpleTypeless() throws InitializationException, IOException {
final TestRunner runner = TestRunners.newTestRunner(FilterCSVColumns.class);
String sql = "select first_name, last_name, company_name, address, city from CSV.A where city='New York'";
runner.setProperty(FilterCSVColumns.SQL_SELECT, sql);
runner.enqueue(Paths.get("src/test/resources/TestFilterCSVColumns/US500_typeless.csv"));
runner.run();
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS);
for (final MockFlowFile flowFile : flowFiles) {
System.out.println(flowFile);
System.out.println(new String(flowFile.toByteArray()));
}
}
}

Просмотреть файл

@ -0,0 +1,379 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.standard;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.MapRecord;
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.RecordSet;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Assert;
import org.junit.Test;
public class TestQueryFlowFile {
static {
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.standard.SQLTransform", "debug");
}
private static final String REL_NAME = "success";
@Test
public void testSimple() throws InitializationException, IOException, SQLException {
final MockRecordParser parser = new MockRecordParser();
parser.addSchemaField("name", RecordFieldType.STRING);
parser.addSchemaField("age", RecordFieldType.INT);
parser.addRecord("Tom", 49);
final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
runner.addControllerService("parser", parser);
runner.enableControllerService(parser);
runner.addControllerService("writer", writer);
runner.enableControllerService(writer);
runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''");
runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
final int numIterations = 1;
for (int i = 0; i < numIterations; i++) {
runner.enqueue(new byte[0]);
}
runner.setThreadCount(4);
runner.run(2 * numIterations);
runner.assertTransferCount(REL_NAME, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
System.out.println(new String(out.toByteArray()));
out.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"49\"\n");
}
@Test
public void testParseFailure() throws InitializationException, IOException, SQLException {
final MockRecordParser parser = new MockRecordParser();
parser.addSchemaField("name", RecordFieldType.STRING);
parser.addSchemaField("age", RecordFieldType.INT);
parser.addRecord("Tom", 49);
final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
runner.addControllerService("parser", parser);
runner.enableControllerService(parser);
runner.addControllerService("writer", writer);
runner.enableControllerService(writer);
runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''");
runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
final int numIterations = 1;
for (int i = 0; i < numIterations; i++) {
runner.enqueue(new byte[0]);
}
runner.setThreadCount(4);
runner.run(2 * numIterations);
runner.assertTransferCount(REL_NAME, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
System.out.println(new String(out.toByteArray()));
out.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"49\"\n");
}
@Test
public void testTransformCalc() throws InitializationException, IOException, SQLException {
final MockRecordParser parser = new MockRecordParser();
parser.addSchemaField("ID", RecordFieldType.INT);
parser.addSchemaField("AMOUNT1", RecordFieldType.FLOAT);
parser.addSchemaField("AMOUNT2", RecordFieldType.FLOAT);
parser.addSchemaField("AMOUNT3", RecordFieldType.FLOAT);
parser.addRecord("008", 10.05F, 15.45F, 89.99F);
parser.addRecord("100", 20.25F, 25.25F, 45.25F);
parser.addRecord("105", 20.05F, 25.05F, 45.05F);
parser.addRecord("200", 34.05F, 25.05F, 75.05F);
final MockRecordWriter writer = new MockRecordWriter("\"NAME\",\"POINTS\"");
final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
runner.addControllerService("parser", parser);
runner.enableControllerService(parser);
runner.addControllerService("writer", writer);
runner.enableControllerService(writer);
runner.setProperty(REL_NAME, "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from FLOWFILE where ID=100");
runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
runner.enqueue(new byte[0]);
runner.run();
runner.assertTransferCount(REL_NAME, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(REL_NAME).get(0);
out.assertContentEquals("\"NAME\",\"POINTS\"\n\"100\",\"90.75\"\n");
}
@Test
public void testAggregateFunction() throws InitializationException, IOException {
final MockRecordParser parser = new MockRecordParser();
parser.addSchemaField("name", RecordFieldType.STRING);
parser.addSchemaField("points", RecordFieldType.INT);
parser.addRecord("Tom", 1);
parser.addRecord("Jerry", 2);
parser.addRecord("Tom", 99);
final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\"");
final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
runner.addControllerService("parser", parser);
runner.enableControllerService(parser);
runner.addControllerService("writer", writer);
runner.enableControllerService(writer);
runner.setProperty(REL_NAME, "select name, sum(points) as points from FLOWFILE GROUP BY name");
runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
runner.enqueue("");
runner.run();
runner.assertTransferCount(REL_NAME, 1);
final MockFlowFile flowFileOut = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS).get(0);
flowFileOut.assertContentEquals("\"name\",\"points\"\n\"Tom\",\"100\"\n\"Jerry\",\"2\"\n");
}
@Test
public void testColumnNames() throws InitializationException, IOException {
final MockRecordParser parser = new MockRecordParser();
parser.addSchemaField("name", RecordFieldType.STRING);
parser.addSchemaField("points", RecordFieldType.INT);
parser.addSchemaField("greeting", RecordFieldType.STRING);
parser.addRecord("Tom", 1, "Hello");
parser.addRecord("Jerry", 2, "Hi");
parser.addRecord("Tom", 99, "Howdy");
final List<String> colNames = new ArrayList<>();
colNames.add("name");
colNames.add("points");
colNames.add("greeting");
colNames.add("FAV_GREETING");
final ResultSetValidatingRecordWriter writer = new ResultSetValidatingRecordWriter(colNames);
final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class);
runner.addControllerService("parser", parser);
runner.enableControllerService(parser);
runner.addControllerService("writer", writer);
runner.enableControllerService(writer);
runner.setProperty(REL_NAME, "select *, greeting AS FAV_GREETING from FLOWFILE");
runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser");
runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer");
runner.enqueue("");
runner.run();
runner.assertTransferCount(REL_NAME, 1);
}
private static class ResultSetValidatingRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
private final List<String> columnNames;
public ResultSetValidatingRecordWriter(final List<String> colNames) {
this.columnNames = new ArrayList<>(colNames);
}
@Override
public RecordSetWriter createWriter(ComponentLog logger) {
return new RecordSetWriter() {
@Override
public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
final int colCount = rs.getSchema().getFieldCount();
Assert.assertEquals(columnNames.size(), colCount);
final List<String> colNames = new ArrayList<>(colCount);
for (int i = 0; i < colCount; i++) {
colNames.add(rs.getSchema().getField(i).getFieldName());
}
Assert.assertEquals(columnNames, colNames);
return WriteResult.of(0, Collections.emptyMap());
}
@Override
public String getMimeType() {
return "text/plain";
}
@Override
public WriteResult write(Record record, OutputStream out) throws IOException {
return null;
}
};
}
}
private static class MockRecordWriter extends AbstractControllerService implements RecordSetWriterFactory {
private final String header;
public MockRecordWriter(final String header) {
this.header = header;
}
@Override
public RecordSetWriter createWriter(final ComponentLog logger) {
return new RecordSetWriter() {
@Override
public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException {
out.write(header.getBytes());
out.write("\n".getBytes());
int recordCount = 0;
final int numCols = rs.getSchema().getFieldCount();
Record record = null;
while ((record = rs.next()) != null) {
recordCount++;
int i = 0;
for (final String fieldName : record.getSchema().getFieldNames()) {
final String val = record.getAsString(fieldName);
out.write("\"".getBytes());
out.write(val.getBytes());
out.write("\"".getBytes());
if (i++ < numCols - 1) {
out.write(",".getBytes());
}
}
out.write("\n".getBytes());
}
return WriteResult.of(recordCount, Collections.emptyMap());
}
@Override
public String getMimeType() {
return "text/plain";
}
@Override
public WriteResult write(Record record, OutputStream out) throws IOException {
return null;
}
};
}
}
private static class MockRecordParser extends AbstractControllerService implements RowRecordReaderFactory {
private final List<Object[]> records = new ArrayList<>();
private final List<RecordField> fields = new ArrayList<>();
private final int failAfterN;
public MockRecordParser() {
this(-1);
}
public MockRecordParser(final int failAfterN) {
this.failAfterN = failAfterN;
}
public void addSchemaField(final String fieldName, final RecordFieldType type) {
fields.add(new RecordField(fieldName, type.getDataType()));
}
public void addRecord(Object... values) {
records.add(values);
}
@Override
public RecordReader createRecordReader(InputStream in, ComponentLog logger) throws IOException {
final Iterator<Object[]> itr = records.iterator();
return new RecordReader() {
private int recordCount = 0;
@Override
public void close() throws IOException {
}
@Override
public Record nextRecord() throws IOException, MalformedRecordException {
if (failAfterN >= recordCount) {
throw new MalformedRecordException("Intentional Unit Test Exception because " + recordCount + " records have been read");
}
recordCount++;
if (!itr.hasNext()) {
return null;
}
final Object[] values = itr.next();
final Map<String, Object> valueMap = new HashMap<>();
int i = 0;
for (final RecordField field : fields) {
final String fieldName = field.getFieldName();
valueMap.put(fieldName, values[i++]);
}
return new MapRecord(new SimpleRecordSchema(fields), valueMap);
}
@Override
public RecordSchema getSchema() {
return new SimpleRecordSchema(fields);
}
};
}
}
}

Просмотреть файл

@ -1,5 +0,0 @@
ID:int,AMOUNT1: float,AMOUNT2:float,AMOUNT3:float
008, 10.05, 15.45, 89.99
100, 20.25, 25.25, 45.25
105, 20.05, 25.05, 45.05
200, 34.05, 25.05, 75.05
1 ID:int AMOUNT1: float AMOUNT2:float AMOUNT3:float
2 008 10.05 15.45 89.99
3 100 20.25 25.25 45.25
4 105 20.05 25.05 45.05
5 200 34.05 25.05 75.05

Различия файлов скрыты, потому что одна или несколько строк слишком длинны

Различия файлов скрыты, потому что одна или несколько строк слишком длинны

Просмотреть файл

@ -0,0 +1,31 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<!--
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.
-->
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-standard-services</artifactId>
<version>1.1.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-record-serialization-service-api</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
</dependencies>
</project>

Просмотреть файл

@ -0,0 +1,82 @@
/*
* 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;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
public class DataTypeValidator implements Validator {
private static final Set<String> validValues;
private static final Set<String> allowsFormatting;
static {
final Set<String> values = new HashSet<>();
values.add("string");
values.add("boolean");
values.add("byte");
values.add("char");
values.add("int");
values.add("long");
values.add("float");
values.add("double");
values.add("time");
values.add("date");
values.add("timestamp");
validValues = Collections.unmodifiableSet(values);
final Set<String> formattable = new HashSet<>();
formattable.add("date");
formattable.add("time");
formattable.add("timestmap");
allowsFormatting = Collections.unmodifiableSet(formattable);
}
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
final String[] splits = input.split("\\:");
final boolean valid;
if (splits.length == 2) {
final String type = splits[0].trim();
if (validValues.contains(type)) {
if (allowsFormatting.contains(splits[0].trim())) {
valid = true;
} else {
valid = false;
}
} else {
valid = false;
}
} else {
valid = validValues.contains(input.trim());
}
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(valid)
.explanation("Valid values for this property are: " + validValues
+ ", where date, time, and timestamp may optionally contain a format (e.g., date:MM-dd-yyyy)")
.build();
}
}

Просмотреть файл

@ -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;
/**
* An Exception that can be thrown to indicate that data was read but could not properly be parsed
*/
public class MalformedRecordException extends Exception {
public MalformedRecordException(final String message) {
super(message);
}
public MalformedRecordException(final String message, final Throwable cause) {
super(message, cause);
}
}

Просмотреть файл

@ -0,0 +1,55 @@
/*
* 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;
import java.io.Closeable;
import java.io.IOException;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
/**
* <p>
* A RowRecordReader is responsible for parsing data and returning a record at a time
* in order to allow the caller to iterate over the records individually.
* </p>
*
* <p>
* PLEASE NOTE: This interface is still considered 'unstable' and may change in a non-backward-compatible
* manner between minor or incremental releases of NiFi.
* </p>
*/
public interface RecordReader extends Closeable {
/**
* Returns the next record in the stream or <code>null</code> if no more records are available.
*
* @param schema the schema to use in order to determine how to interprets the fields in a record
* @return the next record in the stream or <code>null</code> if no more records are available.
*
* @throws IOException if unable to read from the underlying data
* @throws MalformedRecordException if an unrecoverable failure occurs when trying to parse a record
*/
Record nextRecord() throws IOException, MalformedRecordException;
/**
* @return a RecordSchema that is appropriate for the records in the stream
* @throws MalformedRecordException if an unrecoverable failure occurs when trying to parse the underlying data
*/
RecordSchema getSchema() throws MalformedRecordException;
}

Просмотреть файл

@ -0,0 +1,45 @@
/*
* 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;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.nifi.serialization.record.RecordSet;
/**
* <p>
* A ResultSetWriter is responsible for writing a ResultSet to a given {@link OutputStream}.
* </p>
*
* <p>
* PLEASE NOTE: This interface is still considered 'unstable' and may change in a non-backward-compatible
* manner between minor or incremental releases of NiFi.
* </p>
*/
public interface RecordSetWriter extends RecordWriter {
/**
* Writes the given result set to the given output stream
*
* @param recordSet the record set to serialize
* @param out the OutputStream to write to
* @return the results of writing the data
* @throws IOException if unable to write to the given OutputStream
*/
WriteResult write(RecordSet recordSet, OutputStream out) throws IOException;
}

Просмотреть файл

@ -0,0 +1,30 @@
/*
* 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;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.logging.ComponentLog;
/**
* <p>
* A Controller Service that is responsible for creating a {@link RecordSetWriter}.
* </p>
*/
public interface RecordSetWriterFactory extends ControllerService {
RecordSetWriter createWriter(ComponentLog logger);
}

Просмотреть файл

@ -0,0 +1,41 @@
/*
* 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;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.nifi.serialization.record.Record;
public interface RecordWriter {
/**
* Writes the given result set to the given output stream
*
* @param recordSet the record set to serialize
* @param out the OutputStream to write to
* @return the results of writing the data
* @throws IOException if unable to write to the given OutputStream
*/
WriteResult write(Record record, OutputStream out) throws IOException;
/**
* @return the MIME Type that the Result Set Writer produces. This will be added to FlowFiles using
* the mime.type attribute.
*/
String getMimeType();
}

Просмотреть файл

@ -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;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.logging.ComponentLog;
/**
* <p>
* A Controller Service that is responsible for creating a {@link RecordReader}.
* </p>
*/
public interface RowRecordReaderFactory extends ControllerService {
RecordReader createRecordReader(InputStream in, ComponentLog logger) throws MalformedRecordException, IOException;
}

Просмотреть файл

@ -0,0 +1,126 @@
/*
* 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;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.stream.Collectors;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.RecordField;
import org.apache.nifi.serialization.record.RecordSchema;
public class SimpleRecordSchema implements RecordSchema {
private final List<RecordField> fields;
private final Map<String, Integer> fieldIndices;
public SimpleRecordSchema(final List<RecordField> fields) {
this.fields = Collections.unmodifiableList(new ArrayList<>(fields));
this.fieldIndices = new HashMap<>(fields.size());
int index = 0;
for (final RecordField field : fields) {
fieldIndices.put(field.getFieldName(), index++);
}
}
@Override
public List<RecordField> getFields() {
return fields;
}
@Override
public int getFieldCount() {
return fields.size();
}
@Override
public RecordField getField(final int index) {
return fields.get(index);
}
@Override
public List<DataType> getDataTypes() {
return getFields().stream().map(recordField -> recordField.getDataType())
.collect(Collectors.toList());
}
@Override
public List<String> getFieldNames() {
return getFields().stream().map(recordField -> recordField.getFieldName())
.collect(Collectors.toList());
}
@Override
public Optional<DataType> getDataType(final String fieldName) {
final OptionalInt idx = getFieldIndex(fieldName);
return idx.isPresent() ? Optional.of(fields.get(idx.getAsInt()).getDataType()) : Optional.empty();
}
private OptionalInt getFieldIndex(final String fieldName) {
final Integer index = fieldIndices.get(fieldName);
return index == null ? OptionalInt.empty() : OptionalInt.of(index);
}
@Override
public boolean equals(final Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (!(obj instanceof RecordSchema)) {
return false;
}
final RecordSchema other = (RecordSchema) obj;
return fields.equals(other.getFields());
}
@Override
public int hashCode() {
return 143 + 3 * fields.hashCode();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("[");
for (int i = 0; i < fields.size(); i++) {
final RecordField field = fields.get(i);
sb.append("\"");
sb.append(field.getFieldName());
sb.append("\" : \"");
sb.append(field.getDataType());
sb.append("\"");
if (i < fields.size() - 1) {
sb.append(", ");
}
}
sb.append("]");
return sb.toString();
}
}

Просмотреть файл

@ -0,0 +1,69 @@
/*
* 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;
import java.util.Collections;
import java.util.Map;
/**
* <p>
* Provides information about what was written to an OutputStream by a {@link RecordSetWriter}.
* Instances of WriteResult are typically instantiated by calling the static method {@link WriteResult#of(int, Map)}
* or using {@link WriteResult#EMPTY}.
* </p>
*
* <p>
* PLEASE NOTE: This interface is still considered 'unstable' and may change in a non-backward-compatible
* manner between minor or incremental releases of NiFi.
* </p>
*/
public interface WriteResult {
/**
* @return the number of records written
*/
int getRecordCount();
/**
* @return values that should be added to the FlowFile as attributes
*/
Map<String, String> getAttributes();
/**
* Creates a WriteResult with the given record count and attributes
*
* @param recordCount the number of records written
* @param attributes the attributes to add to the FlowFile
* @return A {@link WriteResult} representing the given parameters
*/
public static WriteResult of(final int recordCount, final Map<String, String> attributes) {
return new WriteResult() {
@Override
public int getRecordCount() {
return recordCount;
}
@Override
public Map<String, String> getAttributes() {
return attributes;
}
};
}
public static final WriteResult EMPTY = of(0, Collections.emptyMap());
}

Просмотреть файл

@ -0,0 +1,95 @@
/*
* 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;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
public class DataType {
private final RecordFieldType fieldType;
private final String format;
private final RecordSchema childSchema;
private final List<DataType> childTypes;
DataType(final RecordFieldType fieldType, final String format) {
this(fieldType, format, (RecordSchema) null);
}
DataType(final RecordFieldType fieldType, final String format, final RecordSchema childSchema) {
this.fieldType = fieldType;
this.format = format;
this.childSchema = childSchema;
this.childTypes = Collections.emptyList();
}
DataType(final RecordFieldType fieldType, final String format, final List<DataType> childTypes) {
this.fieldType = fieldType;
this.format = format;
this.childSchema = null;
this.childTypes = Collections.unmodifiableList(childTypes);
}
public String getFormat() {
return format;
}
public RecordFieldType getFieldType() {
return fieldType;
}
public Optional<RecordSchema> getChildRecordSchema() {
return Optional.ofNullable(childSchema);
}
public List<DataType> getPossibleTypes() {
return childTypes;
}
@Override
public int hashCode() {
return 31 + 41 * fieldType.hashCode() + 41 * (format == null ? 0 : format.hashCode());
}
@Override
public boolean equals(final Object obj) {
if (obj == this) {
return true;
}
if (obj == null) {
return false;
}
if (!(obj instanceof DataType)) {
return false;
}
final DataType other = (DataType) obj;
return fieldType.equals(other.fieldType) && ((format == null && other.format == null) || (format != null && format.equals(other.format)));
}
@Override
public String toString() {
if (format == null) {
return fieldType.toString();
} else {
return fieldType.toString() + ":" + format;
}
}
}

Просмотреть файл

@ -0,0 +1,44 @@
/*
* 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;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
public class ListRecordSet implements RecordSet {
private final Iterator<Record> recordItr;
private final RecordSchema schema;
public ListRecordSet(final RecordSchema schema, final List<Record> records) {
this.schema = schema;
final List<Record> copy = new ArrayList<>(records);
recordItr = copy.iterator();
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Record next() {
return recordItr.hasNext() ? recordItr.next() : null;
}
}

Просмотреть файл

@ -0,0 +1,322 @@
/*
* 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;
import java.sql.Time;
import java.sql.Timestamp;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
public class MapRecord implements Record {
private final RecordSchema schema;
private final Map<String, Object> values;
public MapRecord(final RecordSchema schema, final Map<String, Object> values) {
this.schema = Objects.requireNonNull(schema);
this.values = Objects.requireNonNull(values);
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Object[] getValues() {
final Object[] values = new Object[schema.getFieldCount()];
int i = 0;
for (final String fieldName : schema.getFieldNames()) {
values[i++] = getValue(fieldName);
}
return values;
}
@Override
public Object getValue(final String fieldName) {
return values.get(fieldName);
}
@Override
public String getAsString(final String fieldName) {
final Optional<DataType> dataTypeOption = schema.getDataType(fieldName);
if (!dataTypeOption.isPresent()) {
return null;
}
return convertToString(getValue(fieldName), dataTypeOption.get().getFormat());
}
@Override
public String getAsString(final String fieldName, final String format) {
return convertToString(getValue(fieldName), format);
}
private String getFormat(final String optionalFormat, final RecordFieldType fieldType) {
return (optionalFormat == null) ? fieldType.getDefaultFormat() : optionalFormat;
}
private String convertToString(final Object value, final String format) {
if (value == null) {
return null;
}
if (value instanceof java.sql.Date) {
java.sql.Date date = (java.sql.Date) value;
final long time = date.getTime();
return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).format(new java.util.Date(time));
}
if (value instanceof java.util.Date) {
return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).format((java.util.Date) value);
}
if (value instanceof Timestamp) {
java.sql.Timestamp date = (java.sql.Timestamp) value;
final long time = date.getTime();
return new SimpleDateFormat(getFormat(format, RecordFieldType.TIMESTAMP)).format(new java.util.Date(time));
}
if (value instanceof Time) {
java.sql.Time date = (java.sql.Time) value;
final long time = date.getTime();
return new SimpleDateFormat(getFormat(format, RecordFieldType.TIME)).format(new java.util.Date(time));
}
return value.toString();
}
@Override
public Long getAsLong(final String fieldName) {
return convertToLong(getValue(fieldName), fieldName);
}
private Long convertToLong(final Object value, final Object fieldDesc) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).longValue();
}
if (value instanceof String) {
return Long.parseLong((String) value);
}
if (value instanceof Date) {
return ((Date) value).getTime();
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Long for field " + fieldDesc);
}
@Override
public Integer getAsInt(final String fieldName) {
return convertToInt(getValue(fieldName), fieldName);
}
private Integer convertToInt(final Object value, final Object fieldDesc) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).intValue();
}
if (value instanceof String) {
return Integer.parseInt((String) value);
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Integer for field " + fieldDesc);
}
@Override
public Double getAsDouble(final String fieldName) {
return convertToDouble(getValue(fieldName), fieldName);
}
private Double convertToDouble(final Object value, final Object fieldDesc) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).doubleValue();
}
if (value instanceof String) {
return Double.parseDouble((String) value);
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Double for field " + fieldDesc);
}
@Override
public Float getAsFloat(final String fieldName) {
return convertToFloat(getValue(fieldName), fieldName);
}
private Float convertToFloat(final Object value, final Object fieldDesc) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).floatValue();
}
if (value instanceof String) {
return Float.parseFloat((String) value);
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Float for field " + fieldDesc);
}
@Override
public Record getAsRecord(String fieldName) {
return convertToRecord(getValue(fieldName), fieldName);
}
private Record convertToRecord(final Object value, final Object fieldDesc) {
if (value == null) {
return null;
}
if (value instanceof Record) {
return (Record) value;
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Record for field " + fieldDesc);
}
@Override
public Boolean getAsBoolean(final String fieldName) {
return convertToBoolean(getValue(fieldName), fieldName);
}
private Boolean convertToBoolean(final Object value, final Object fieldDesc) {
if (value == null) {
return null;
}
if (value instanceof Boolean) {
return (Boolean) value;
}
if (value instanceof String) {
final String string = (String) value;
if (string.equalsIgnoreCase("true") || string.equalsIgnoreCase("t")) {
return Boolean.TRUE;
}
if (string.equalsIgnoreCase("false") || string.equals("f")) {
return Boolean.FALSE;
}
throw new TypeMismatchException("Cannot convert String value to Boolean for field " + fieldDesc + " because it is not a valid boolean value");
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Boolean for field " + fieldDesc);
}
@Override
public Date getAsDate(final String fieldName) {
final Optional<DataType> dataTypeOption = schema.getDataType(fieldName);
if (!dataTypeOption.isPresent()) {
return null;
}
return convertToDate(getValue(fieldName), fieldName, dataTypeOption.get().getFormat());
}
@Override
public Date getAsDate(final String fieldName, final String format) {
return convertToDate(getValue(fieldName), fieldName, format);
}
private Date convertToDate(final Object value, final Object fieldDesc, final String format) {
if (value == null) {
return null;
}
if (value instanceof Date) {
return (Date) value;
}
if (value instanceof Number) {
final Long time = ((Number) value).longValue();
return new Date(time);
}
if (value instanceof java.sql.Date) {
return new Date(((java.sql.Date) value).getTime());
}
if (value instanceof String) {
try {
return new SimpleDateFormat(getFormat(format, RecordFieldType.DATE)).parse((String) value);
} catch (final ParseException e) {
throw new TypeMismatchException("Cannot convert String value to date for field " + fieldDesc + " because it is not in the correct format of: " + format, e);
}
}
throw new TypeMismatchException("Cannot convert value of type " + value.getClass() + " to Boolean for field " + fieldDesc);
}
@Override
public Object[] getAsArray(final String fieldName) {
return convertToArray(getValue(fieldName));
}
private Object[] convertToArray(final Object value) {
if (value == null) {
return null;
}
if (value instanceof Object[]) {
return (Object[]) value;
}
if (value instanceof List) {
return ((List<?>) value).toArray();
}
return new Object[] {value};
}
@Override
public int hashCode() {
return 31 + 41 * values.hashCode() + 7 * schema.hashCode();
}
@Override
public boolean equals(final Object obj) {
if (obj == this) {
return true;
}
if (obj == null) {
return false;
}
if (!(obj instanceof MapRecord)) {
return false;
}
final MapRecord other = (MapRecord) obj;
return schema.equals(other.schema) && values.equals(other.values);
}
@Override
public String toString() {
return "MapRecord[values=" + values + "]";
}
}

Просмотреть файл

@ -0,0 +1,62 @@
/*
* 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;
import java.util.Date;
public interface Record {
RecordSchema getSchema();
/**
* <p>
* Returns a view of the the values of the fields in this Record.
* </p>
*
* <b>NOTE:</b> The array that is returned may be an underlying array that is backing
* the contents of the Record. As such, modifying the array in any way may result in
* modifying the record.
*
* @return a view of the values of the fields in this Record
*/
Object[] getValues();
Object getValue(String fieldName);
String getAsString(String fieldName);
String getAsString(String fieldName, String format);
Long getAsLong(String fieldName);
Integer getAsInt(String fieldName);
Double getAsDouble(String fieldName);
Float getAsFloat(String fieldName);
Record getAsRecord(String fieldName);
Boolean getAsBoolean(String fieldName);
Date getAsDate(String fieldName);
Date getAsDate(String fieldName, String format);
Object[] getAsArray(String fieldName);
}

Просмотреть файл

@ -0,0 +1,64 @@
/*
* 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;
public class RecordField {
private final String fieldName;
private final DataType dataType;
public RecordField(final String fieldName, final DataType dataType) {
this.fieldName = fieldName;
this.dataType = dataType;
}
public String getFieldName() {
return fieldName;
}
public DataType getDataType() {
return dataType;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + ((dataType == null) ? 0 : dataType.hashCode());
result = prime * result + ((fieldName == null) ? 0 : fieldName.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
RecordField other = (RecordField) obj;
return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName());
}
}

Просмотреть файл

@ -0,0 +1,114 @@
/*
* 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;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public enum RecordFieldType {
STRING("string"),
BOOLEAN("boolean"),
BYTE("byte"),
CHAR("char"),
SHORT("short"),
INT("int"),
BIGINT("bigint"),
LONG("long"),
FLOAT("float"),
DOUBLE("double"),
DATE("date", "yyyy-MM-dd"),
TIME("time", "HH:mm:ss"),
TIMESTAMP("timestamp", "yyyy-MM-dd HH:mm:ss"),
RECORD("record"),
CHOICE("choice"),
ARRAY("array");
private static final Map<String, RecordFieldType> SIMPLE_NAME_MAP = new HashMap<String, RecordFieldType>();
static {
for (RecordFieldType value : values()) {
SIMPLE_NAME_MAP.put(value.simpleName, value);
}
}
private final String simpleName;
private final String defaultFormat;
private final DataType defaultDataType;
private RecordFieldType(final String simpleName) {
this(simpleName, null);
}
private RecordFieldType(final String simpleName, final String defaultFormat) {
this.simpleName = simpleName;
this.defaultFormat = defaultFormat;
this.defaultDataType = new DataType(this, defaultFormat);
}
public String getDefaultFormat() {
return defaultFormat;
}
/**
* @return the DataType with the default format
*/
public DataType getDataType() {
return defaultDataType;
}
public DataType getDataType(final String format) {
return new DataType(this, format);
}
/**
* Returns a Data Type that represents a "RECORD" type with the given schema.
*
* @param childSchema the Schema for the Record
* @return a DataType that represents a Record with the given schema, or <code>null</code> if this RecordFieldType
* is not the RECORD type.
*/
public DataType getDataType(final RecordSchema childSchema) {
if (this != RECORD) {
return null;
}
return new DataType(this, getDefaultFormat(), childSchema);
}
/**
* Returns a Data Type that represents a "CHOICE" of multiple possible types. This method is
* only applicable for a RecordFieldType of {@link #CHOICE}.
*
* @param possibleChildTypes the possible types that are allowable
* @return a DataType that represents a "CHOICE" of multiple possible types, or <code>null</code> if this RecordFieldType
* is not the CHOICE type.
*/
public DataType getDataType(final List<DataType> possibleChildTypes) {
if (this != CHOICE) {
return null;
}
return new DataType(this, getDefaultFormat(), possibleChildTypes);
}
public static RecordFieldType of(final String typeString) {
return SIMPLE_NAME_MAP.get(typeString);
}
}

Просмотреть файл

@ -0,0 +1,58 @@
/*
* 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;
import java.util.List;
import java.util.Optional;
public interface RecordSchema {
/**
* @return the list of fields that are present in the schema
*/
List<RecordField> getFields();
/**
* @return the number of fields in the schema
*/
int getFieldCount();
/**
* @param index the 0-based index of which field to return
* @return the index'th field
*
* @throws IndexOutOfBoundsException if the index is < 0 or >= the number of fields (determined by {@link #getFieldCount()}).
*/
RecordField getField(int index);
/**
* @return the data types of the fields
*/
List<DataType> getDataTypes();
/**
* @return the names of the fields
*/
List<String> getFieldNames();
/**
* @param fieldName the name of the field whose type is desired
* @return the RecordFieldType associated with the field that has the given name, or
* <code>null</code> if the schema does not contain a field with the given name
*/
Optional<DataType> getDataType(String fieldName);
}

Просмотреть файл

@ -0,0 +1,53 @@
/*
* 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;
import java.io.IOException;
public interface RecordSet {
/**
* @return the {@link RecordSchema} that applies to the records in this RecordSet
*/
RecordSchema getSchema() throws IOException;
/**
* @return the next {@link Record} in the set or <code>null</code> if there are no more records
*/
Record next() throws IOException;
public static RecordSet of(final RecordSchema schema, final Record... records) {
return new RecordSet() {
private int index = 0;
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Record next() {
if (index >= records.length) {
return null;
}
return records[index++];
}
};
}
}

Просмотреть файл

@ -0,0 +1,169 @@
/*
* 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;
import java.io.Closeable;
import java.io.IOException;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.sql.Types;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ResultSetRecordSet implements RecordSet, Closeable {
private static final Logger logger = LoggerFactory.getLogger(ResultSetRecordSet.class);
private final ResultSet rs;
private final RecordSchema schema;
private final Set<String> rsColumnNames;
public ResultSetRecordSet(final ResultSet rs) throws SQLException {
this.rs = rs;
this.schema = createSchema(rs);
rsColumnNames = new HashSet<>();
final ResultSetMetaData metadata = rs.getMetaData();
for (int i = 0; i < metadata.getColumnCount(); i++) {
rsColumnNames.add(metadata.getColumnLabel(i + 1));
}
}
@Override
public RecordSchema getSchema() {
return schema;
}
@Override
public Record next() throws IOException {
try {
if (rs.next()) {
return createRecord(rs);
}
} catch (final SQLException e) {
throw new IOException("Could not obtain next record from ResultSet", e);
}
return null;
}
@Override
public void close() {
try {
rs.close();
} catch (final SQLException e) {
logger.error("Failed to close ResultSet", e);
}
}
private Record createRecord(final ResultSet rs) throws SQLException {
final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
for (final RecordField field : schema.getFields()) {
final String fieldName = field.getFieldName();
final Object value;
if (rsColumnNames.contains(fieldName)) {
value = rs.getObject(field.getFieldName());
} else {
value = null;
}
values.put(fieldName, value);
}
return new MapRecord(schema, values);
}
private static RecordSchema createSchema(final ResultSet rs) throws SQLException {
final ResultSetMetaData metadata = rs.getMetaData();
final int numCols = metadata.getColumnCount();
final List<RecordField> fields = new ArrayList<>(numCols);
for (int i = 0; i < numCols; i++) {
final int column = i + 1;
final int sqlType = metadata.getColumnType(column);
final RecordFieldType fieldType = getFieldType(sqlType);
final String fieldName = metadata.getColumnLabel(column);
final RecordField field = new RecordField(fieldName, fieldType.getDataType());
fields.add(field);
}
return new SimpleRecordSchema(fields);
}
private static RecordFieldType getFieldType(final int sqlType) {
switch (sqlType) {
case Types.ARRAY:
return RecordFieldType.ARRAY;
case Types.BIGINT:
case Types.ROWID:
return RecordFieldType.LONG;
case Types.BINARY:
case Types.LONGVARBINARY:
case Types.VARBINARY:
return RecordFieldType.ARRAY;
case Types.BIT:
case Types.BOOLEAN:
return RecordFieldType.BOOLEAN;
case Types.CHAR:
return RecordFieldType.CHAR;
case Types.DATE:
return RecordFieldType.DATE;
case Types.DECIMAL:
case Types.DOUBLE:
case Types.NUMERIC:
case Types.REAL:
return RecordFieldType.DOUBLE;
case Types.FLOAT:
return RecordFieldType.FLOAT;
case Types.INTEGER:
return RecordFieldType.INT;
case Types.SMALLINT:
return RecordFieldType.SHORT;
case Types.TINYINT:
return RecordFieldType.BYTE;
case Types.LONGNVARCHAR:
case Types.LONGVARCHAR:
case Types.NCHAR:
case Types.NULL:
case Types.NVARCHAR:
case Types.VARCHAR:
return RecordFieldType.STRING;
case Types.OTHER:
case Types.JAVA_OBJECT:
return RecordFieldType.RECORD;
case Types.TIME:
case Types.TIME_WITH_TIMEZONE:
return RecordFieldType.TIME;
case Types.TIMESTAMP:
case Types.TIMESTAMP_WITH_TIMEZONE:
return RecordFieldType.TIMESTAMP;
}
return RecordFieldType.STRING;
}
}

Просмотреть файл

@ -0,0 +1,28 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.serialization.record;
public class TypeMismatchException extends RuntimeException {
public TypeMismatchException(String message) {
super(message);
}
public TypeMismatchException(String message, Throwable cause) {
super(message, cause);
}
}

Просмотреть файл

@ -0,0 +1,41 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<!--
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.
-->
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-services-bundle</artifactId>
<version>1.2.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-record-serialization-services-nar</artifactId>
<packaging>nar</packaging>
<properties>
<maven.javadoc.skip>true</maven.javadoc.skip>
<source.skip>true</source.skip>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-standard-services-api-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-services</artifactId>
<version>1.2.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
</dependencies>
</project>

Просмотреть файл

@ -0,0 +1,269 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.
APACHE NIFI SUBCOMPONENTS:
The Apache NiFi project contains subcomponents with separate copyright
notices and license terms. Your use of the source code for the these
subcomponents is subject to the terms and conditions of the following
licenses.
The binary distribution of this product bundles 'Hamcrest' which is available
under a BSD license. More details found here: http://hamcrest.org.
Copyright (c) 2000-2006, www.hamcrest.org
All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are met:
Redistributions of source code must retain the above copyright notice, this list of
conditions and the following disclaimer. Redistributions in binary form must reproduce
the above copyright notice, this list of conditions and the following disclaimer in
the documentation and/or other materials provided with the distribution.
Neither the name of Hamcrest nor the names of its contributors may be used to endorse
or promote products derived from this software without specific prior written
permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
DAMAGE.
This product bundles 'asm' which is available under a 3-Clause BSD style license.
For details see http://asm.ow2.org/asmdex-license.html
Copyright (c) 2012 France Télécom
All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions
are met:
1. Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
2. Redistributions in binary form must reproduce the above copyright
notice, this list of conditions and the following disclaimer in the
documentation and/or other materials provided with the distribution.
3. Neither the name of the copyright holders nor the names of its
contributors may be used to endorse or promote products derived from
this software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
THE POSSIBILITY OF SUCH DAMAGE.

Просмотреть файл

@ -0,0 +1,77 @@
nifi-record-serialization-services-nar
Copyright 2014-2017 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
===========================================
Apache Software License v2
===========================================
The following binary components are provided under the Apache Software License v2
(ASLv2) Apache Commons Lang
The following NOTICE information applies:
Apache Commons Lang
Copyright 2001-2015 The Apache Software Foundation
This product includes software from the Spring Framework,
under the Apache License 2.0 (see: StringUtils.containsWhitespace())
(ASLv2) Grok
The following NOTICE information applies:
Grok
Copyright 2014 Anthony Corbacho, and contributors.
(ASLv2) Groovy (org.codehaus.groovy:groovy:jar:2.4.5 - http://www.groovy-lang.org)
The following NOTICE information applies:
Groovy Language
Copyright 2003-2015 The respective authors and developers
Developers and Contributors are listed in the project POM file
and Gradle build file
This product includes software developed by
The Groovy community (http://groovy.codehaus.org/).
(ASLv2) Google GSON
The following NOTICE information applies:
Copyright 2008 Google Inc.
(ASLv2) Jackson JSON processor
The following NOTICE information applies:
# Jackson JSON processor
Jackson is a high-performance, Free/Open Source JSON processing library.
It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
been in development since 2007.
It is currently developed by a community of developers, as well as supported
commercially by FasterXML.com.
## Licensing
Jackson core and extension components may licensed under different licenses.
To find the details that apply to this artifact see the accompanying LICENSE file.
For more information, including possible other licensing options, contact
FasterXML.com (http://fasterxml.com).
## Credits
A list of contributors may be found from CREDITS file, which is included
in some artifacts (usually source distributions); but is always available
from the source code management (SCM) system project uses.
(ASLv2) JSON-SMART
The following NOTICE information applies:
Copyright 2011 JSON-SMART authors
(ASLv2) JsonPath
The following NOTICE information applies:
Copyright 2011 JsonPath authors
(ASLv2) opencsv (net.sf.opencsv:opencsv:2.3)
(ASLv2) Apache Avro
The following NOTICE information applies:
Apache Avro
Copyright 2009-2013 The Apache Software Foundation

Просмотреть файл

@ -0,0 +1 @@
/bin/

Просмотреть файл

@ -0,0 +1,94 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<!-- 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. -->
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-services-bundle</artifactId>
<version>1.2.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-record-serialization-services</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-record-serialization-service-api</artifactId>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>net.sf.opencsv</groupId>
<artifactId>opencsv</artifactId>
<version>2.3</version>
</dependency>
<dependency>
<groupId>io.thekraken</groupId>
<artifactId>grok</artifactId>
<version>0.1.5</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
<configuration>
<excludes combine.children="append">
<exclude>src/test/resources/csv/extra-white-space.csv</exclude>
<exclude>src/test/resources/csv/multi-bank-account.csv</exclude>
<exclude>src/test/resources/csv/single-bank-account.csv</exclude>
<exclude>src/test/resources/grok/error-with-stack-trace.log</exclude>
<exclude>src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log</exclude>
<exclude>src/test/resources/grok/nifi-log-sample.log</exclude>
<exclude>src/test/resources/grok/single-line-log-messages.txt</exclude>
<exclude>src/test/resources/json/bank-account-array-different-schemas.json</exclude>
<exclude>src/test/resources/json/bank-account-array.json</exclude>
<exclude>src/test/resources/json/json-with-unicode.json</exclude>
<exclude>src/test/resources/json/primitive-type-array.json</exclude>
<exclude>src/test/resources/json/single-bank-account.json</exclude>
<exclude>src/test/resources/json/single-element-nested-array.json</exclude>
<exclude>src/test/resources/json/single-element-nested.json</exclude>
</excludes>
</configuration>
</plugin>
</plugins>
</build>
</project>

Просмотреть файл

@ -0,0 +1,40 @@
/*
* 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.avro;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
@Tags({"avro", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"})
@CapabilityDescription("Parses Avro data and returns each Avro record as an separate record.")
public class AvroReader extends AbstractControllerService implements RowRecordReaderFactory {
@Override
public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException {
return new AvroRecordReader(in);
}
}

Просмотреть файл

@ -0,0 +1,254 @@
/*
* 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.avro;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.Schema.Type;
import org.apache.avro.file.DataFileStream;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.Array;
import org.apache.avro.generic.GenericData.StringType;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.util.Utf8;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
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;
public class AvroRecordReader implements RecordReader {
private final InputStream in;
private final Schema avroSchema;
private final DataFileStream<GenericRecord> dataFileStream;
private RecordSchema recordSchema;
public AvroRecordReader(final InputStream in) throws IOException, MalformedRecordException {
this.in = in;
dataFileStream = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>());
this.avroSchema = dataFileStream.getSchema();
GenericData.setStringType(this.avroSchema, StringType.String);
}
@Override
public void close() throws IOException {
dataFileStream.close();
in.close();
}
@Override
public Record nextRecord() throws IOException, MalformedRecordException {
if (!dataFileStream.hasNext()) {
return null;
}
GenericRecord record = null;
while (record == null && dataFileStream.hasNext()) {
record = dataFileStream.next();
}
final RecordSchema schema = getSchema();
final Map<String, Object> values = convertRecordToObjectArray(record, schema);
return new MapRecord(schema, values);
}
private Map<String, Object> convertRecordToObjectArray(final GenericRecord record, final RecordSchema schema) {
final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
for (final String fieldName : schema.getFieldNames()) {
final Object value = record.get(fieldName);
final Field avroField = record.getSchema().getField(fieldName);
if (avroField == null) {
values.put(fieldName, null);
continue;
}
final Schema fieldSchema = avroField.schema();
final DataType dataType = schema.getDataType(fieldName).orElse(null);
final Object converted = convertValue(value, fieldSchema, avroField.name(), dataType);
values.put(fieldName, converted);
}
return values;
}
@Override
public RecordSchema getSchema() throws MalformedRecordException {
if (recordSchema != null) {
return recordSchema;
}
recordSchema = createSchema(avroSchema);
return recordSchema;
}
private RecordSchema createSchema(final Schema avroSchema) {
final List<RecordField> recordFields = new ArrayList<>(avroSchema.getFields().size());
for (final Field field : avroSchema.getFields()) {
final String fieldName = field.name();
final DataType dataType = determineDataType(field.schema());
recordFields.add(new RecordField(fieldName, dataType));
}
final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
return recordSchema;
}
private Object convertValue(final Object value, final Schema avroSchema, final String fieldName, final DataType desiredType) {
if (value == null) {
return null;
}
switch (avroSchema.getType()) {
case UNION:
if (value instanceof GenericData.Record) {
final GenericData.Record record = (GenericData.Record) value;
return convertValue(value, record.getSchema(), fieldName, desiredType);
}
break;
case RECORD:
final GenericData.Record record = (GenericData.Record) value;
final Schema recordSchema = record.getSchema();
final List<Field> recordFields = recordSchema.getFields();
final Map<String, Object> values = new HashMap<>(recordFields.size());
for (final Field field : recordFields) {
final DataType desiredFieldType = determineDataType(field.schema());
final Object avroFieldValue = record.get(field.name());
final Object fieldValue = convertValue(avroFieldValue, field.schema(), field.name(), desiredFieldType);
values.put(field.name(), fieldValue);
}
final RecordSchema childSchema = createSchema(recordSchema);
return new MapRecord(childSchema, values);
case BYTES:
final ByteBuffer bb = (ByteBuffer) value;
return bb.array();
case FIXED:
final GenericFixed fixed = (GenericFixed) value;
return fixed.bytes();
case ENUM:
return value.toString();
case NULL:
return null;
case STRING:
return value.toString();
case ARRAY:
final Array<?> array = (Array<?>) value;
final Object[] valueArray = new Object[array.size()];
for (int i = 0; i < array.size(); i++) {
final Schema elementSchema = avroSchema.getElementType();
valueArray[i] = convertValue(array.get(i), elementSchema, fieldName, determineDataType(elementSchema));
}
return valueArray;
case MAP:
final Map<?, ?> avroMap = (Map<?, ?>) value;
final Map<String, Object> map = new HashMap<>(avroMap.size());
for (final Map.Entry<?, ?> entry : avroMap.entrySet()) {
Object obj = entry.getValue();
if (obj instanceof Utf8 || obj instanceof CharSequence) {
obj = obj.toString();
}
map.put(entry.getKey().toString(), obj);
}
return map;
}
return value;
}
private DataType determineDataType(final Schema avroSchema) {
final Type avroType = avroSchema.getType();
switch (avroType) {
case ARRAY:
case BYTES:
case FIXED:
return RecordFieldType.ARRAY.getDataType();
case BOOLEAN:
return RecordFieldType.BOOLEAN.getDataType();
case DOUBLE:
return RecordFieldType.DOUBLE.getDataType();
case ENUM:
case STRING:
return RecordFieldType.STRING.getDataType();
case FLOAT:
return RecordFieldType.FLOAT.getDataType();
case INT:
return RecordFieldType.INT.getDataType();
case LONG:
return RecordFieldType.LONG.getDataType();
case RECORD: {
final List<Field> avroFields = avroSchema.getFields();
final List<RecordField> recordFields = new ArrayList<>(avroFields.size());
for (final Field field : avroFields) {
final String fieldName = field.name();
final Schema fieldSchema = field.schema();
final DataType fieldType = determineDataType(fieldSchema);
recordFields.add(new RecordField(fieldName, fieldType));
}
final RecordSchema recordSchema = new SimpleRecordSchema(recordFields);
return RecordFieldType.RECORD.getDataType(recordSchema);
}
case NULL:
case MAP:
return RecordFieldType.RECORD.getDataType();
case UNION: {
final List<Schema> nonNullSubSchemas = avroSchema.getTypes().stream()
.filter(s -> s.getType() != Type.NULL)
.collect(Collectors.toList());
if (nonNullSubSchemas.size() == 1) {
return determineDataType(nonNullSubSchemas.get(0));
}
final List<DataType> possibleChildTypes = new ArrayList<>(nonNullSubSchemas.size());
for (final Schema subSchema : nonNullSubSchemas) {
final DataType childDataType = determineDataType(subSchema);
possibleChildTypes.add(childDataType);
}
return RecordFieldType.CHOICE.getDataType(possibleChildTypes);
}
}
return null;
}
}

Просмотреть файл

@ -0,0 +1,67 @@
/*
* 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.avro;
import java.util.ArrayList;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.AbstractRecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@Tags({"avro", "result", "set", "writer", "serializer", "record", "row"})
@CapabilityDescription("Writes the contents of a Database ResultSet in Binary Avro format. The data types in the Result Set must match those "
+ "specified by the Avro Schema. No type coercion will occur, with the exception of Date, Time, and Timestamps fields because Avro does not provide "
+ "support for these types specifically. As a result, they will be converted to String fields using the configured formats. In addition, the label"
+ "of the column must be a valid Avro field name.")
public class AvroRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
.name("Avro Schema")
.description("The Avro Schema to use when writing out the Result Set")
.addValidator(new AvroSchemaValidator())
.expressionLanguageSupported(false)
.required(true)
.build();
private volatile Schema schema;
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
properties.add(SCHEMA);
return properties;
}
@OnEnabled
public void storePropertyValues(final ConfigurationContext context) {
schema = new Schema.Parser().parse(context.getProperty(SCHEMA).getValue());
}
@Override
public RecordSetWriter createWriter(final ComponentLog logger) {
return new WriteAvroResult(schema, getDateFormat(), getTimeFormat(), getTimestampFormat());
}
}

Просмотреть файл

@ -0,0 +1,45 @@
/*
* 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.avro;
import org.apache.avro.Schema;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
public class AvroSchemaValidator implements Validator {
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
try {
new Schema.Parser().parse(input);
return new ValidationResult.Builder()
.valid(true)
.build();
} catch (final Exception e) {
return new ValidationResult.Builder()
.input(input)
.subject(subject)
.valid(false)
.explanation("Not a valid Avro Schema: " + e.getMessage())
.build();
}
}
}

Просмотреть файл

@ -0,0 +1,286 @@
/*
* 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.avro;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.sql.Blob;
import java.sql.Clob;
import java.sql.SQLException;
import java.sql.Time;
import java.sql.Timestamp;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
import org.apache.nifi.serialization.record.RecordSet;
public class WriteAvroResult implements RecordSetWriter {
private final Schema schema;
private final DateFormat dateFormat;
private final DateFormat timeFormat;
private final DateFormat timestampFormat;
public WriteAvroResult(final Schema schema, final String dateFormat, final String timeFormat, final String timestampFormat) {
this.schema = schema;
this.dateFormat = new SimpleDateFormat(dateFormat);
this.timeFormat = new SimpleDateFormat(timeFormat);
this.timestampFormat = new SimpleDateFormat(timestampFormat);
}
@Override
public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException {
Record record = rs.next();
if (record == null) {
return WriteResult.of(0, Collections.emptyMap());
}
final GenericRecord rec = new GenericData.Record(schema);
int nrOfRows = 0;
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
dataFileWriter.create(schema, outStream);
final RecordSchema recordSchema = rs.getSchema();
do {
for (final String fieldName : recordSchema.getFieldNames()) {
final Object value = record.getValue(fieldName);
final Field field = schema.getField(fieldName);
if (field == null) {
continue;
}
final Object converted;
try {
converted = convert(value, field.schema(), fieldName);
} catch (final SQLException e) {
throw new IOException("Failed to write records to stream", e);
}
rec.put(fieldName, converted);
}
dataFileWriter.append(rec);
nrOfRows++;
} while ((record = rs.next()) != null);
}
return WriteResult.of(nrOfRows, Collections.emptyMap());
}
@Override
public WriteResult write(final Record record, final OutputStream out) throws IOException {
final GenericRecord rec = new GenericData.Record(schema);
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
dataFileWriter.create(schema, out);
final RecordSchema recordSchema = record.getSchema();
for (final String fieldName : recordSchema.getFieldNames()) {
final Object value = record.getValue(fieldName);
final Field field = schema.getField(fieldName);
if (field == null) {
continue;
}
final Object converted;
try {
converted = convert(value, field.schema(), fieldName);
} catch (final SQLException e) {
throw new IOException("Failed to write records to stream", e);
}
rec.put(fieldName, converted);
}
dataFileWriter.append(rec);
}
return WriteResult.of(1, Collections.emptyMap());
}
private Object convert(final Object value, final Schema schema, final String fieldName) throws SQLException, IOException {
if (value == null) {
return null;
}
// Need to handle CLOB and BLOB before getObject() is called, due to ResultSet's maximum portability statement
if (value instanceof Clob) {
final Clob clob = (Clob) value;
long numChars = clob.length();
char[] buffer = new char[(int) numChars];
InputStream is = clob.getAsciiStream();
int index = 0;
int c = is.read();
while (c > 0) {
buffer[index++] = (char) c;
c = is.read();
}
clob.free();
return new String(buffer);
}
if (value instanceof Blob) {
final Blob blob = (Blob) value;
final long numChars = blob.length();
final byte[] buffer = new byte[(int) numChars];
final InputStream is = blob.getBinaryStream();
int index = 0;
int c = is.read();
while (c > 0) {
buffer[index++] = (byte) c;
c = is.read();
}
final ByteBuffer bb = ByteBuffer.wrap(buffer);
blob.free();
return bb;
}
if (value instanceof byte[]) {
// bytes requires little bit different handling
return ByteBuffer.wrap((byte[]) value);
} else if (value instanceof Byte) {
// tinyint(1) type is returned by JDBC driver as java.sql.Types.TINYINT
// But value is returned by JDBC as java.lang.Byte
// (at least H2 JDBC works this way)
// direct put to avro record results:
// org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
return ((Byte) value).intValue();
} else if (value instanceof Short) {
//MS SQL returns TINYINT as a Java Short, which Avro doesn't understand.
return ((Short) value).intValue();
} else if (value instanceof BigDecimal) {
// Avro can't handle BigDecimal as a number - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
return value.toString();
} else if (value instanceof BigInteger) {
// Check the precision of the BIGINT. Some databases allow arbitrary precision (> 19), but Avro won't handle that.
// It the SQL type is BIGINT and the precision is between 0 and 19 (inclusive); if so, the BigInteger is likely a
// long (and the schema says it will be), so try to get its value as a long.
// Otherwise, Avro can't handle BigInteger as a number - it will throw an AvroRuntimeException
// such as: "Unknown datum type: java.math.BigInteger: 38". In this case the schema is expecting a string.
final BigInteger bigInt = (BigInteger) value;
if (bigInt.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
return value.toString();
} else {
return bigInt.longValue();
}
} else if (value instanceof Boolean) {
return value;
} else if (value instanceof Map) {
// TODO: Revisit how we handle a lot of these cases....
switch (schema.getType()) {
case MAP:
return value;
case RECORD:
final GenericData.Record avroRecord = new GenericData.Record(schema);
final Record record = (Record) value;
for (final String recordFieldName : record.getSchema().getFieldNames()) {
final Object recordFieldValue = record.getValue(recordFieldName);
final Field field = schema.getField(recordFieldName);
if (field == null) {
continue;
}
final Object converted = convert(recordFieldValue, field.schema(), recordFieldName);
avroRecord.put(recordFieldName, converted);
}
return avroRecord;
}
return value.toString();
} else if (value instanceof List) {
return value;
} else if (value instanceof Object[]) {
final List<Object> list = new ArrayList<>();
for (final Object o : ((Object[]) value)) {
final Object converted = convert(o, schema.getElementType(), fieldName);
list.add(converted);
}
return list;
} else if (value instanceof Number) {
return value;
} else if (value instanceof java.util.Date) {
final java.util.Date date = (java.util.Date) value;
return dateFormat.format(date);
} else if (value instanceof java.sql.Date) {
final java.sql.Date sqlDate = (java.sql.Date) value;
final java.util.Date date = new java.util.Date(sqlDate.getTime());
return dateFormat.format(date);
} else if (value instanceof Time) {
final Time time = (Time) value;
final java.util.Date date = new java.util.Date(time.getTime());
return timeFormat.format(date);
} else if (value instanceof Timestamp) {
final Timestamp time = (Timestamp) value;
final java.util.Date date = new java.util.Date(time.getTime());
return timestampFormat.format(date);
}
// The different types that we support are numbers (int, long, double, float),
// as well as boolean values and Strings. Since Avro doesn't provide
// timestamp types, we want to convert those to Strings. So we will cast anything other
// than numbers or booleans to strings by using the toString() method.
return value.toString();
}
@Override
public String getMimeType() {
return "application/avro-binary";
}
public static String normalizeNameForAvro(String inputName) {
String normalizedName = inputName.replaceAll("[^A-Za-z0-9_]", "_");
if (Character.isDigit(normalizedName.charAt(0))) {
normalizedName = "_" + normalizedName;
}
return normalizedName;
}
}

Просмотреть файл

@ -0,0 +1,49 @@
/*
* 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.csv;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.UserTypeOverrideRowReader;
@Tags({"csv", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"})
@CapabilityDescription("Parses CSV-formatted data, returning each row in the CSV file as a separate record. "
+ "This reader assumes that the first line in the content is the column names and all subsequent lines are "
+ "the values. By default, the reader will assume that all columns are of 'String' type, but this can be "
+ "overridden by adding a user-defined Property where the key is the name of a column and the value is the "
+ "type of the column. For example, if a Property has the name \"balance\" with a value of float, it the "
+ "reader will attempt to coerce all values in the \"balance\" column into a floating-point number. See "
+ "Controller Service's Usage for further documentation.")
@DynamicProperty(name = "<name of column in CSV>", value = "<type of column values in CSV>",
description = "User-defined properties are used to indicate that the values of a specific column should be interpreted as a "
+ "user-defined data type (e.g., int, double, float, date, etc.)", supportsExpressionLanguage = false)
public class CSVReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
@Override
public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException {
return new CSVRecordReader(in, logger, getFieldTypeOverrides());
}
}

Просмотреть файл

@ -0,0 +1,216 @@
/*
* 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.csv;
import java.io.BufferedInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
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 au.com.bytecode.opencsv.CSVReader;
public class CSVRecordReader implements RecordReader {
private final ComponentLog logger;
private final CSVReader reader;
private final String[] firstLine;
private final Map<String, DataType> fieldTypeOverrides;
private RecordSchema schema;
public CSVRecordReader(final InputStream in, final ComponentLog logger, final Map<String, DataType> fieldTypeOverrides) throws IOException {
this.logger = logger;
reader = new CSVReader(new InputStreamReader(new BufferedInputStream(in)));
firstLine = reader.readNext();
this.fieldTypeOverrides = fieldTypeOverrides;
}
@Override
public Record nextRecord() throws IOException, MalformedRecordException {
final RecordSchema schema = getSchema();
while (true) {
final String[] line = reader.readNext();
if (line == null) {
return null;
}
final List<DataType> fieldTypes = schema.getDataTypes();
if (fieldTypes.size() != line.length) {
logger.warn("Found record with incorrect number of fields. Expected {} but found {}; skipping record", new Object[] {fieldTypes.size(), line.length});
continue;
}
try {
final Map<String, Object> rowValues = new HashMap<>(schema.getFieldCount());
int i = 0;
for (final String fieldName : schema.getFieldNames()) {
if (i >= line.length) {
rowValues.put(fieldName, null);
continue;
}
final String rawValue = line[i++].trim();
final Object converted = convert(schema.getDataType(fieldName).orElse(null), rawValue);
rowValues.put(fieldName, converted);
}
return new MapRecord(schema, rowValues);
} catch (final Exception e) {
throw new MalformedRecordException("Found invalid CSV record", e);
}
}
}
@Override
public RecordSchema getSchema() {
if (schema != null) {
return schema;
}
final List<RecordField> recordFields = new ArrayList<>();
for (final String element : firstLine) {
final String name = element.trim();
final DataType dataType;
final DataType overriddenDataType = fieldTypeOverrides.get(name);
if (overriddenDataType != null) {
dataType = overriddenDataType;
} else {
dataType = RecordFieldType.STRING.getDataType();
}
final RecordField field = new RecordField(name, dataType);
recordFields.add(field);
}
if (recordFields.isEmpty()) {
recordFields.add(new RecordField("line", RecordFieldType.STRING.getDataType()));
}
schema = new SimpleRecordSchema(recordFields);
return schema;
}
protected Object convert(final DataType dataType, final String value) {
if (dataType == null) {
return value;
}
switch (dataType.getFieldType()) {
case BOOLEAN:
if (value.length() == 0) {
return null;
}
return Boolean.parseBoolean(value);
case BYTE:
if (value.length() == 0) {
return null;
}
return Byte.parseByte(value);
case SHORT:
if (value.length() == 0) {
return null;
}
return Short.parseShort(value);
case INT:
if (value.length() == 0) {
return null;
}
return Integer.parseInt(value);
case LONG:
case BIGINT:
if (value.length() == 0) {
return null;
}
return Long.parseLong(value);
case FLOAT:
if (value.length() == 0) {
return null;
}
return Float.parseFloat(value);
case DOUBLE:
if (value.length() == 0) {
return null;
}
return Double.parseDouble(value);
case DATE:
if (value.length() == 0) {
return null;
}
try {
final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
return new java.sql.Date(date.getTime());
} catch (final ParseException e) {
logger.warn("Found invalid value for DATE field: " + value + " does not match expected format of "
+ dataType.getFormat() + "; will substitute a NULL value for this field");
return null;
}
case TIME:
if (value.length() == 0) {
return null;
}
try {
final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
return new java.sql.Time(date.getTime());
} catch (final ParseException e) {
logger.warn("Found invalid value for TIME field: " + value + " does not match expected format of "
+ dataType.getFormat() + "; will substitute a NULL value for this field");
return null;
}
case TIMESTAMP:
if (value.length() == 0) {
return null;
}
try {
final Date date = new SimpleDateFormat(dataType.getFormat()).parse(value);
return new java.sql.Timestamp(date.getTime());
} catch (final ParseException e) {
logger.warn("Found invalid value for TIMESTAMP field: " + value + " does not match expected format of "
+ dataType.getFormat() + "; will substitute a NULL value for this field");
return null;
}
case STRING:
default:
return value;
}
}
@Override
public void close() throws IOException {
reader.close();
}
}

Просмотреть файл

@ -0,0 +1,37 @@
/*
* 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.csv;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.AbstractRecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@Tags({"csv", "result", "set", "writer", "serializer", "record", "row"})
@CapabilityDescription("Writes the contents of a Database ResultSet as CSV data. The first line written "
+ "will be the column names. All subsequent lines will be the values corresponding to those columns.")
public class CSVRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
@Override
public RecordSetWriter createWriter(final ComponentLog logger) {
return new WriteCSVResult(getDateFormat(), getTimeFormat(), getTimestampFormat());
}
}

Просмотреть файл

@ -0,0 +1,127 @@
/*
* 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.csv;
import java.io.IOException;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.util.Collections;
import java.util.Optional;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
import org.apache.nifi.serialization.record.RecordSet;
import org.apache.nifi.stream.io.NonCloseableOutputStream;
import au.com.bytecode.opencsv.CSVWriter;
public class WriteCSVResult implements RecordSetWriter {
private final String dateFormat;
private final String timeFormat;
private final String timestampFormat;
public WriteCSVResult(final String dateFormat, final String timeFormat, final String timestampFormat) {
this.dateFormat = dateFormat;
this.timeFormat = timeFormat;
this.timestampFormat = timestampFormat;
}
private String getFormat(final Record record, final String fieldName) {
final Optional<DataType> dataTypeOption = record.getSchema().getDataType(fieldName);
if (!dataTypeOption.isPresent()) {
return null;
}
final DataType dataType = dataTypeOption.get();
switch (dataType.getFieldType()) {
case DATE:
return dateFormat == null ? dataType.getFormat() : dateFormat;
case TIME:
return timeFormat == null ? dataType.getFormat() : timeFormat;
case TIMESTAMP:
return timestampFormat == null ? dataType.getFormat() : timestampFormat;
}
return dataType.getFormat();
}
@Override
public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException {
int count = 0;
try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
final CSVWriter writer = new CSVWriter(streamWriter)) {
try {
final RecordSchema schema = rs.getSchema();
final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
writer.writeNext(columnNames);
Record record;
while ((record = rs.next()) != null) {
final String[] colVals = new String[schema.getFieldCount()];
int i = 0;
for (final String fieldName : schema.getFieldNames()) {
colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
}
writer.writeNext(colVals);
count++;
}
} catch (final Exception e) {
throw new IOException("Failed to serialize results", e);
}
}
return WriteResult.of(count, Collections.emptyMap());
}
@Override
public WriteResult write(final Record record, final OutputStream rawOut) throws IOException {
try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut);
final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable);
final CSVWriter writer = new CSVWriter(streamWriter)) {
try {
final RecordSchema schema = record.getSchema();
final String[] columnNames = schema.getFieldNames().toArray(new String[0]);
writer.writeNext(columnNames);
final String[] colVals = new String[schema.getFieldCount()];
int i = 0;
for (final String fieldName : schema.getFieldNames()) {
colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName));
}
writer.writeNext(colVals);
} catch (final Exception e) {
throw new IOException("Failed to serialize results", e);
}
}
return WriteResult.of(1, Collections.emptyMap());
}
@Override
public String getMimeType() {
return "text/csv";
}
}

Просмотреть файл

@ -0,0 +1,48 @@
/*
* 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.grok;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import io.thekraken.grok.api.Grok;
public class GrokExpressionValidator implements Validator {
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
try {
new Grok().compile(input);
} catch (final Exception e) {
return new ValidationResult.Builder()
.input(input)
.subject(subject)
.valid(false)
.explanation("Invalid Grok pattern: " + e.getMessage())
.build();
}
return new ValidationResult.Builder()
.input(input)
.subject(subject)
.valid(true)
.build();
}
}

Просмотреть файл

@ -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.grok;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.Reader;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.UserTypeOverrideRowReader;
import io.thekraken.grok.api.Grok;
import io.thekraken.grok.api.exception.GrokException;
@Tags({"grok", "logs", "logfiles", "parse", "unstructured", "text", "record", "reader", "regex", "pattern", "logstash"})
@CapabilityDescription("Provides a mechanism for reading unstructured text data, such as log files, and structuring the data "
+ "so that it can be processed. The service is configured using Grok patterns. "
+ "The service reads from a stream of data and splits each message that it finds into a separate Record, each containing the fields that are configured. "
+ "If a line in the input does not match the expected message pattern, the line of text is considered to be part of the previous "
+ "message, with the exception of stack traces. A stack trace that is found at the end of a log message is considered to be part "
+ "of the previous message but is added to the 'STACK_TRACE' field of the Record. If a record has no stack trace, it will have a NULL value "
+ "for the STACK_TRACE field.")
public class GrokReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
private volatile Grok grok;
private static final String DEFAULT_PATTERN_NAME = "/default-grok-patterns.txt";
static final PropertyDescriptor PATTERN_FILE = new PropertyDescriptor.Builder()
.name("Grok Pattern File")
.description("Path to a file that contains Grok Patterns to use for parsing logs. If not specified, a built-in default Pattern file "
+ "will be used. If specified, all patterns in the given pattern file will override the default patterns. See the Controller Service's "
+ "Additional Details for a list of pre-defined patterns.")
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.expressionLanguageSupported(true)
.required(false)
.build();
static final PropertyDescriptor GROK_EXPRESSION = new PropertyDescriptor.Builder()
.name("Grok Expression")
.description("Specifies the format of a log line in Grok format. This allows the Record Reader to understand how to parse each log line. "
+ "If a line in the log file does not match this pattern, the line will be assumed to belong to the previous log message.")
.addValidator(new GrokExpressionValidator())
.required(true)
.build();
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(PATTERN_FILE);
properties.add(GROK_EXPRESSION);
return properties;
}
@OnEnabled
public void preCompile(final ConfigurationContext context) throws GrokException, IOException {
grok = new Grok();
try (final InputStream in = getClass().getResourceAsStream(DEFAULT_PATTERN_NAME);
final Reader reader = new InputStreamReader(in)) {
grok.addPatternFromReader(reader);
}
if (context.getProperty(PATTERN_FILE).isSet()) {
grok.addPatternFromFile(context.getProperty(PATTERN_FILE).getValue());
}
grok.compile(context.getProperty(GROK_EXPRESSION).getValue());
}
@Override
public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException {
return new GrokRecordReader(in, grok, getFieldTypeOverrides());
}
}

Просмотреть файл

@ -0,0 +1,323 @@
/*
* 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.grok;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.text.ParseException;
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.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.lang3.time.FastDateFormat;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
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 io.thekraken.grok.api.Grok;
import io.thekraken.grok.api.GrokUtils;
import io.thekraken.grok.api.Match;
public class GrokRecordReader implements RecordReader {
private final BufferedReader reader;
private final Grok grok;
private final Map<String, DataType> fieldTypeOverrides;
private String nextLine;
private RecordSchema schema;
static final String STACK_TRACE_COLUMN_NAME = "STACK_TRACE";
private static final Pattern STACK_TRACE_PATTERN = Pattern.compile(
"^\\s*(?:(?: |\\t)+at )|"
+ "(?:(?: |\\t)+\\[CIRCULAR REFERENCE\\:)|"
+ "(?:Caused by\\: )|"
+ "(?:Suppressed\\: )|"
+ "(?:\\s+... \\d+ (?:more|common frames? omitted)$)");
private static final FastDateFormat TIME_FORMAT_DATE;
private static final FastDateFormat TIME_FORMAT_TIME;
private static final FastDateFormat TIME_FORMAT_TIMESTAMP;
static {
final TimeZone gmt = TimeZone.getTimeZone("GMT");
TIME_FORMAT_DATE = FastDateFormat.getInstance("yyyy-MM-dd", gmt);
TIME_FORMAT_TIME = FastDateFormat.getInstance("HH:mm:ss", gmt);
TIME_FORMAT_TIMESTAMP = FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt);
}
public GrokRecordReader(final InputStream in, final Grok grok, final Map<String, DataType> fieldTypeOverrides) {
this.reader = new BufferedReader(new InputStreamReader(in));
this.grok = grok;
this.fieldTypeOverrides = fieldTypeOverrides;
}
@Override
public void close() throws IOException {
reader.close();
}
@Override
public Record nextRecord() throws IOException, MalformedRecordException {
final String line = nextLine == null ? reader.readLine() : nextLine;
nextLine = null; // ensure that we don't process nextLine again
if (line == null) {
return null;
}
final RecordSchema schema = getSchema();
final Match match = grok.match(line);
match.captures();
final Map<String, Object> valueMap = match.toMap();
if (valueMap.isEmpty()) { // We were unable to match the pattern so return an empty Object array.
return new MapRecord(schema, Collections.emptyMap());
}
// Read the next line to see if it matches the pattern (in which case we will simply leave it for
// the next call to nextRecord()) or we will attach it to the previously read record.
String stackTrace = null;
final StringBuilder toAppend = new StringBuilder();
while ((nextLine = reader.readLine()) != null) {
final Match nextLineMatch = grok.match(nextLine);
nextLineMatch.captures();
final Map<String, Object> nextValueMap = nextLineMatch.toMap();
if (nextValueMap.isEmpty()) {
// next line did not match. Check if it indicates a Stack Trace. If so, read until
// the stack trace ends. Otherwise, append the next line to the last field in the record.
if (isStartOfStackTrace(nextLine)) {
stackTrace = readStackTrace(nextLine);
break;
} else {
toAppend.append("\n").append(nextLine);
}
} else {
// The next line matched our pattern.
break;
}
}
try {
final List<DataType> fieldTypes = schema.getDataTypes();
final Map<String, Object> values = new HashMap<>(fieldTypes.size());
for (final String fieldName : schema.getFieldNames()) {
final Object value = valueMap.get(fieldName);
if (value == null) {
values.put(fieldName, null);
continue;
}
final DataType fieldType = schema.getDataType(fieldName).orElse(null);
final Object converted = convert(fieldType, value.toString());
values.put(fieldName, converted);
}
final String lastFieldBeforeStackTrace = schema.getFieldNames().get(schema.getFieldCount() - 2);
if (toAppend.length() > 0) {
final Object existingValue = values.get(lastFieldBeforeStackTrace);
final String updatedValue = existingValue == null ? toAppend.toString() : existingValue + toAppend.toString();
values.put(lastFieldBeforeStackTrace, updatedValue);
}
values.put(STACK_TRACE_COLUMN_NAME, stackTrace);
return new MapRecord(schema, values);
} catch (final Exception e) {
throw new MalformedRecordException("Found invalid log record and will skip it. Record: " + line, e);
}
}
private boolean isStartOfStackTrace(final String line) {
if (line == null) {
return false;
}
// Stack Traces are generally of the form:
// java.lang.IllegalArgumentException: My message
// at org.apache.nifi.MyClass.myMethod(MyClass.java:48)
// at java.lang.Thread.run(Thread.java:745) [na:1.8.0_60]
// Caused by: java.net.SocketTimeoutException: null
// ... 13 common frames omitted
int index = line.indexOf("Exception: ");
if (index < 0) {
index = line.indexOf("Error: ");
}
if (index < 0) {
return false;
}
if (line.indexOf(" ") < index) {
return false;
}
return true;
}
private String readStackTrace(final String firstLine) throws IOException {
final StringBuilder sb = new StringBuilder(firstLine);
String line;
while ((line = reader.readLine()) != null) {
if (isLineInStackTrace(line)) {
sb.append("\n").append(line);
} else {
nextLine = line;
break;
}
}
return sb.toString();
}
private boolean isLineInStackTrace(final String line) {
return STACK_TRACE_PATTERN.matcher(line).find();
}
protected Object convert(final DataType fieldType, final String string) {
if (fieldType == null) {
return string;
}
switch (fieldType.getFieldType()) {
case BOOLEAN:
if (string.length() == 0) {
return null;
}
return Boolean.parseBoolean(string);
case BYTE:
if (string.length() == 0) {
return null;
}
return Byte.parseByte(string);
case SHORT:
if (string.length() == 0) {
return null;
}
return Short.parseShort(string);
case INT:
if (string.length() == 0) {
return null;
}
return Integer.parseInt(string);
case LONG:
if (string.length() == 0) {
return null;
}
return Long.parseLong(string);
case FLOAT:
if (string.length() == 0) {
return null;
}
return Float.parseFloat(string);
case DOUBLE:
if (string.length() == 0) {
return null;
}
return Double.parseDouble(string);
case DATE:
if (string.length() == 0) {
return null;
}
try {
Date date = TIME_FORMAT_DATE.parse(string);
return new java.sql.Date(date.getTime());
} catch (ParseException e) {
return null;
}
case TIME:
if (string.length() == 0) {
return null;
}
try {
Date date = TIME_FORMAT_TIME.parse(string);
return new java.sql.Time(date.getTime());
} catch (ParseException e) {
return null;
}
case TIMESTAMP:
if (string.length() == 0) {
return null;
}
try {
Date date = TIME_FORMAT_TIMESTAMP.parse(string);
return new java.sql.Timestamp(date.getTime());
} catch (ParseException e) {
return null;
}
case STRING:
default:
return string;
}
}
@Override
public RecordSchema getSchema() {
if (schema != null) {
return schema;
}
final List<RecordField> fields = new ArrayList<>();
String grokExpression = grok.getOriginalGrokPattern();
while (grokExpression.length() > 0) {
final Matcher matcher = GrokUtils.GROK_PATTERN.matcher(grokExpression);
if (matcher.find()) {
final Map<String, String> namedGroups = GrokUtils.namedGroups(matcher, grokExpression);
final String fieldName = namedGroups.get("subname");
DataType dataType = fieldTypeOverrides.get(fieldName);
if (dataType == null) {
dataType = RecordFieldType.STRING.getDataType();
}
final RecordField recordField = new RecordField(fieldName, dataType);
fields.add(recordField);
if (grokExpression.length() > matcher.end() + 1) {
grokExpression = grokExpression.substring(matcher.end() + 1);
} else {
break;
}
}
}
fields.add(new RecordField(STACK_TRACE_COLUMN_NAME, RecordFieldType.STRING.getDataType()));
schema = new SimpleRecordSchema(fields);
return schema;
}
}

Просмотреть файл

@ -0,0 +1,307 @@
/*
* 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.json;
import java.io.IOException;
import java.io.InputStream;
import java.text.DateFormat;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.TimeZone;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
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.codehaus.jackson.JsonFactory;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.JsonParseException;
import org.codehaus.jackson.JsonParser;
import org.codehaus.jackson.JsonToken;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.node.ArrayNode;
public abstract class AbstractJsonRowRecordReader implements RecordReader {
private final ComponentLog logger;
private final JsonParser jsonParser;
private final JsonFactory jsonFactory;
private final boolean array;
private final JsonNode firstJsonNode;
private boolean firstObjectConsumed = false;
private static final TimeZone gmt = TimeZone.getTimeZone("GMT");
public AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
this.logger = logger;
jsonFactory = new JsonFactory();
try {
jsonParser = jsonFactory.createJsonParser(in);
jsonParser.setCodec(new ObjectMapper());
JsonToken token = jsonParser.nextToken();
if (token == JsonToken.START_ARRAY) {
array = true;
token = jsonParser.nextToken(); // advance to START_OBJECT token
} else {
array = false;
}
if (token == JsonToken.START_OBJECT) { // could be END_ARRAY also
firstJsonNode = jsonParser.readValueAsTree();
} else {
firstJsonNode = null;
}
} catch (final JsonParseException e) {
throw new MalformedRecordException("Could not parse data as JSON", e);
}
}
@Override
public Record nextRecord() throws IOException, MalformedRecordException {
if (firstObjectConsumed && !array) {
return null;
}
final JsonNode nextNode = getNextJsonNode();
final RecordSchema schema = getSchema();
try {
return convertJsonNodeToRecord(nextNode, schema);
} catch (final MalformedRecordException mre) {
throw mre;
} catch (final IOException ioe) {
throw ioe;
} catch (final Exception e) {
logger.debug("Failed to convert JSON Element {} into a Record object using schema {} due to {}", new Object[] {nextNode, schema, e.toString(), e});
throw new MalformedRecordException("Successfully parsed a JSON object from input but failed to convert into a Record object with the given schema", e);
}
}
protected DataType determineFieldType(final JsonNode node) {
if (node.isDouble()) {
return RecordFieldType.DOUBLE.getDataType();
}
if (node.isBoolean()) {
return RecordFieldType.BOOLEAN.getDataType();
}
if (node.isFloatingPointNumber()) {
return RecordFieldType.FLOAT.getDataType();
}
if (node.isBigInteger()) {
return RecordFieldType.BIGINT.getDataType();
}
if (node.isBigDecimal()) {
return RecordFieldType.DOUBLE.getDataType();
}
if (node.isLong()) {
return RecordFieldType.LONG.getDataType();
}
if (node.isInt()) {
return RecordFieldType.INT.getDataType();
}
if (node.isTextual()) {
return RecordFieldType.STRING.getDataType();
}
if (node.isArray()) {
return RecordFieldType.ARRAY.getDataType();
}
final RecordSchema childSchema = determineSchema(node);
return RecordFieldType.RECORD.getDataType(childSchema);
}
protected RecordSchema determineSchema(final JsonNode jsonNode) {
final List<RecordField> recordFields = new ArrayList<>();
final Iterator<Map.Entry<String, JsonNode>> itr = jsonNode.getFields();
while (itr.hasNext()) {
final Map.Entry<String, JsonNode> entry = itr.next();
final String elementName = entry.getKey();
final JsonNode node = entry.getValue();
DataType dataType = determineFieldType(node);
recordFields.add(new RecordField(elementName, dataType));
}
return new SimpleRecordSchema(recordFields);
}
protected Object convertField(final JsonNode fieldNode, final String fieldName, final DataType desiredType) throws IOException, MalformedRecordException {
if (fieldNode == null || fieldNode.isNull()) {
return null;
}
switch (desiredType.getFieldType()) {
case BOOLEAN:
return fieldNode.asBoolean();
case BYTE:
return (byte) fieldNode.asInt();
case CHAR:
final String text = fieldNode.asText();
if (text.isEmpty()) {
return null;
}
return text.charAt(0);
case DOUBLE:
return fieldNode.asDouble();
case FLOAT:
return (float) fieldNode.asDouble();
case INT:
return fieldNode.asInt();
case LONG:
return fieldNode.asLong();
case SHORT:
return (short) fieldNode.asInt();
case STRING:
return fieldNode.asText();
case DATE: {
final String string = fieldNode.asText();
if (string.isEmpty()) {
return null;
}
try {
final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
dateFormat.setTimeZone(gmt);
final Date date = dateFormat.parse(string);
return new java.sql.Date(date.getTime());
} catch (ParseException e) {
logger.warn("Failed to convert JSON field to Date for field {} (value {})", new Object[] {fieldName, string, e});
return null;
}
}
case TIME: {
final String string = fieldNode.asText();
if (string.isEmpty()) {
return null;
}
try {
final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
dateFormat.setTimeZone(gmt);
final Date date = dateFormat.parse(string);
return new java.sql.Date(date.getTime());
} catch (ParseException e) {
logger.warn("Failed to convert JSON field to Time for field {} (value {})", new Object[] {fieldName, string, e});
return null;
}
}
case TIMESTAMP: {
final String string = fieldNode.asText();
if (string.isEmpty()) {
return null;
}
try {
final DateFormat dateFormat = new SimpleDateFormat(desiredType.getFormat());
dateFormat.setTimeZone(gmt);
final Date date = dateFormat.parse(string);
return new java.sql.Date(date.getTime());
} catch (ParseException e) {
logger.warn("Failed to convert JSON field to Timestamp for field {} (value {})", new Object[] {fieldName, string, e});
return null;
}
}
case ARRAY: {
final ArrayNode arrayNode = (ArrayNode) fieldNode;
final int numElements = arrayNode.size();
final Object[] arrayElements = new Object[numElements];
int count = 0;
for (final JsonNode node : arrayNode) {
final Object converted = convertField(node, fieldName, determineFieldType(node));
arrayElements[count++] = converted;
}
return arrayElements;
}
case RECORD: {
if (fieldNode.isObject()) {
final Optional<RecordSchema> childSchema = desiredType.getChildRecordSchema();
if (!childSchema.isPresent()) {
return null;
}
return convertJsonNodeToRecord(fieldNode, childSchema.get());
} else {
return fieldNode.toString();
}
}
}
return fieldNode.toString();
}
private JsonNode getNextJsonNode() throws JsonParseException, IOException, MalformedRecordException {
if (!firstObjectConsumed) {
firstObjectConsumed = true;
return firstJsonNode;
}
while (true) {
final JsonToken token = jsonParser.nextToken();
if (token == null) {
return null;
}
switch (token) {
case END_OBJECT:
continue;
case START_OBJECT:
return jsonParser.readValueAsTree();
case END_ARRAY:
case START_ARRAY:
return null;
default:
throw new MalformedRecordException("Expected to get a JSON Object but got a token of type " + token.name());
}
}
}
@Override
public void close() throws IOException {
jsonParser.close();
}
protected JsonParser getJsonParser() {
return jsonParser;
}
protected JsonFactory getJsonFactory() {
return jsonFactory;
}
protected Optional<JsonNode> getFirstJsonNode() {
return Optional.ofNullable(firstJsonNode);
}
protected abstract Record convertJsonNodeToRecord(final JsonNode nextNode, final RecordSchema schema) throws IOException, MalformedRecordException;
}

Просмотреть файл

@ -0,0 +1,126 @@
/*
* 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.json;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.record.DataType;
import com.jayway.jsonpath.JsonPath;
@Tags({"json", "jsonpath", "record", "reader", "parser"})
@CapabilityDescription("Parses JSON records and evaluates user-defined JSON Path's against each JSON object. The root element may be either "
+ "a single JSON object or a JSON array. If a JSON array is found, each JSON object within that array is treated as a separate record. "
+ "User-defined properties define the fields that should be extracted from the JSON in order to form the fields of a Record. Any JSON field "
+ "that is not extracted via a JSONPath will not be returned in the JSON Records.")
@SeeAlso(JsonTreeReader.class)
@DynamicProperty(name = "The field name for the record. If it is desirable to enforce that the value be coerced into a given type, its type can be included "
+ "in the name by using a syntax of <field name>:<field type>. For example, \"balance:double\".",
value="A JSONPath Expression that will be evaluated against each JSON record. The result of the JSONPath will be the value of the "
+ "field whose name is the same as the property name.",
description="User-defined properties identifiy how to extract specific fields from a JSON object in order to create a Record",
supportsExpressionLanguage=false)
public class JsonPathReader extends AbstractControllerService implements RowRecordReaderFactory {
private volatile LinkedHashMap<String, JsonPath> jsonPaths;
private volatile Map<String, DataType> fieldTypeOverrides;
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.description("JsonPath Expression that indicates how to retrieve the value from a JSON Object for the '" + propertyDescriptorName + "' column")
.dynamic(true)
.required(false)
.addValidator(new JsonPathValidator())
.build();
}
@OnEnabled
public void compileJsonPaths(final ConfigurationContext context) {
final Map<String, DataType> fieldTypes = new HashMap<>(context.getProperties().size());
final LinkedHashMap<String, JsonPath> compiled = new LinkedHashMap<>();
for (final PropertyDescriptor descriptor : context.getProperties().keySet()) {
if (!descriptor.isDynamic()) {
continue;
}
final String fieldName = PropertyNameUtil.getFieldName(descriptor.getName());
final Optional<DataType> dataTypeOption = PropertyNameUtil.getDataType(descriptor.getName());
if (dataTypeOption.isPresent()) {
fieldTypes.put(fieldName, dataTypeOption.get());
}
final String expression = context.getProperty(descriptor).getValue();
final JsonPath jsonPath = JsonPath.compile(expression);
compiled.put(fieldName, jsonPath);
}
jsonPaths = compiled;
fieldTypeOverrides = fieldTypes;
}
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
boolean pathSpecified = false;
for (final PropertyDescriptor property : validationContext.getProperties().keySet()) {
if (property.isDynamic()) {
pathSpecified = true;
break;
}
}
if (pathSpecified) {
return Collections.emptyList();
}
return Collections.singleton(new ValidationResult.Builder()
.subject("JSON Paths")
.valid(false)
.explanation("No JSON Paths were specified")
.build());
}
@Override
public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
return new JsonPathRowRecordReader(jsonPaths, fieldTypeOverrides, in, logger);
}
}

Просмотреть файл

@ -0,0 +1,280 @@
/*
* 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.json;
import java.io.IOException;
import java.io.InputStream;
import java.text.DateFormat;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TimeZone;
import java.util.stream.Collectors;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.DataTypeUtils;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
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.codehaus.jackson.JsonNode;
import com.jayway.jsonpath.Configuration;
import com.jayway.jsonpath.DocumentContext;
import com.jayway.jsonpath.JsonPath;
import com.jayway.jsonpath.PathNotFoundException;
import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
public class JsonPathRowRecordReader extends AbstractJsonRowRecordReader {
private static final Configuration STRICT_PROVIDER_CONFIGURATION = Configuration.builder().jsonProvider(new JacksonJsonProvider()).build();
private static final String TIME_FORMAT_DATE = "yyyy-MM-dd";
private static final String TIME_FORMAT_TIME = "HH:mm:ss";
private static final String TIME_FORMAT_TIMESTAMP = "yyyy-MM-dd HH:mm:ss";
private static final TimeZone gmt = TimeZone.getTimeZone("GMT");
private final LinkedHashMap<String, JsonPath> jsonPaths;
private final Map<String, DataType> fieldTypeOverrides;
private final InputStream in;
private RecordSchema schema;
public JsonPathRowRecordReader(final LinkedHashMap<String, JsonPath> jsonPaths, final Map<String, DataType> fieldTypeOverrides, final InputStream in, final ComponentLog logger)
throws MalformedRecordException, IOException {
super(in, logger);
this.jsonPaths = jsonPaths;
this.fieldTypeOverrides = fieldTypeOverrides;
this.in = in;
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public RecordSchema getSchema() {
if (schema != null) {
return schema;
}
final Optional<JsonNode> firstNodeOption = getFirstJsonNode();
final List<RecordField> recordFields = new ArrayList<>();
if (firstNodeOption.isPresent()) {
final DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(firstNodeOption.get().toString());
for (final Map.Entry<String, JsonPath> entry : jsonPaths.entrySet()) {
final String fieldName = PropertyNameUtil.getFieldName(entry.getKey());
final JsonPath jsonPath = entry.getValue();
final DataType dataType;
final DataType dataTypeOverride = fieldTypeOverrides.get(fieldName);
if (dataTypeOverride == null) {
Object value;
try {
value = ctx.read(jsonPath);
} catch (final PathNotFoundException pnfe) {
value = null;
}
if (value == null) {
dataType = RecordFieldType.STRING.getDataType();
} else {
dataType = DataTypeUtils.inferDataType(value);
}
} else {
dataType = dataTypeOverride;
}
recordFields.add(new RecordField(fieldName, dataType));
}
}
// If there are any overridden field types that we didn't find, add as the last fields.
final Set<String> knownFieldNames = recordFields.stream()
.map(f -> f.getFieldName())
.collect(Collectors.toSet());
for (final Map.Entry<String, DataType> entry : fieldTypeOverrides.entrySet()) {
if (!knownFieldNames.contains(entry.getKey())) {
recordFields.add(new RecordField(entry.getKey(), entry.getValue()));
}
}
schema = new SimpleRecordSchema(recordFields);
return schema;
}
@Override
@SuppressWarnings("unchecked")
protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException {
if (jsonNode == null) {
return null;
}
final DocumentContext ctx = JsonPath.using(STRICT_PROVIDER_CONFIGURATION).parse(jsonNode.toString());
final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
for (final Map.Entry<String, JsonPath> entry : jsonPaths.entrySet()) {
final JsonPath jsonPath = entry.getValue();
Object value;
try {
value = ctx.read(jsonPath);
} catch (final PathNotFoundException pnfe) {
value = null;
}
final String fieldName = entry.getKey();
if (value != null) {
final DataType determinedType = DataTypeUtils.inferDataType(value);
final DataType desiredType = schema.getDataType(fieldName).orElse(null);
if (value instanceof List) {
value = ((List<Object>) value).toArray();
} else if (value instanceof Map && desiredType.getFieldType() == RecordFieldType.RECORD) {
value = convert(desiredType, value);
} else if (desiredType != null && !determinedType.equals(desiredType) && shouldConvert(value, determinedType.getFieldType())) {
value = convert(desiredType, value);
}
}
values.put(fieldName, value);
}
return new MapRecord(schema, values);
}
private boolean shouldConvert(final Object value, final RecordFieldType determinedType) {
return determinedType != null
&& determinedType != RecordFieldType.ARRAY;
}
protected Object convert(final DataType dataType, final Object value) {
if (dataType.getFieldType() == RecordFieldType.RECORD && dataType.getChildRecordSchema().isPresent() && value instanceof Map) {
@SuppressWarnings("unchecked")
final Map<String, Object> map = (Map<String, Object>) value;
return new MapRecord(dataType.getChildRecordSchema().get(), map);
} else {
return convertString(dataType, value.toString());
}
}
/**
* Coerces the given string into the provided data type, if possible
*
* @param dataType the desired type
* @param string the string representation of the value
* @return an Object representing the same value as the given string but in the requested data type
*/
protected Object convertString(final DataType dataType, final String string) {
if (dataType == null) {
return string;
}
switch (dataType.getFieldType()) {
case BOOLEAN:
if (string.length() == 0) {
return null;
}
return Boolean.parseBoolean(string);
case BYTE:
if (string.length() == 0) {
return null;
}
return Byte.parseByte(string);
case SHORT:
if (string.length() == 0) {
return null;
}
return Short.parseShort(string);
case INT:
if (string.length() == 0) {
return null;
}
return Integer.parseInt(string);
case LONG:
if (string.length() == 0) {
return null;
}
return Long.parseLong(string);
case FLOAT:
if (string.length() == 0) {
return null;
}
return Float.parseFloat(string);
case DOUBLE:
if (string.length() == 0) {
return null;
}
return Double.parseDouble(string);
case DATE:
if (string.length() == 0) {
return null;
}
try {
final DateFormat format = new SimpleDateFormat(TIME_FORMAT_DATE);
format.setTimeZone(gmt);
Date date = format.parse(string);
return new java.sql.Date(date.getTime());
} catch (ParseException e) {
return null;
}
case TIME:
if (string.length() == 0) {
return null;
}
try {
final DateFormat format = new SimpleDateFormat(TIME_FORMAT_TIME);
format.setTimeZone(gmt);
Date date = format.parse(string);
return new java.sql.Time(date.getTime());
} catch (ParseException e) {
return null;
}
case TIMESTAMP:
if (string.length() == 0) {
return null;
}
try {
final DateFormat format = new SimpleDateFormat(TIME_FORMAT_TIMESTAMP);
format.setTimeZone(gmt);
Date date = format.parse(string);
return new java.sql.Timestamp(date.getTime());
} catch (ParseException e) {
return null;
}
case STRING:
default:
return string;
}
}
}

Просмотреть файл

@ -0,0 +1,60 @@
/*
* 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.json;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import com.jayway.jsonpath.JsonPath;
public class JsonPathValidator implements Validator {
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
if (PropertyNameUtil.hasFieldType(subject) && !PropertyNameUtil.isFieldTypeValid(subject)) {
final String fieldType = PropertyNameUtil.getFieldTypeName(subject).get();
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(false)
.explanation("Invalid field type. If property name contains a colon (:) it must use syntax of "
+ "<field name>:<field type> but the specified field type ('" + fieldType + "') is not a valid field type")
.build();
}
try {
JsonPath.compile(input);
} catch (final Exception e) {
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(false)
.explanation("Invalid JSON Path Expression: " + e.getMessage())
.build();
}
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(true)
.build();
}
}

Просмотреть файл

@ -0,0 +1,66 @@
/*
* 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.json;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.AbstractRecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@Tags({"json", "resultset", "writer", "serialize", "record", "row"})
@CapabilityDescription("Writes the results of a Database ResultSet as a JSON Array. Even if the ResultSet "
+ "consists of a single row, it will be written as an array with a single element.")
public class JsonRecordSetWriter extends AbstractRecordSetWriter implements RecordSetWriterFactory {
static final PropertyDescriptor PRETTY_PRINT_JSON = new PropertyDescriptor.Builder()
.name("Pretty Print JSON")
.description("Specifies whether or not the JSON should be pretty printed")
.expressionLanguageSupported(false)
.allowableValues("true", "false")
.defaultValue("false")
.required(true)
.build();
private boolean prettyPrint;
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
properties.add(PRETTY_PRINT_JSON);
return properties;
}
@OnEnabled
public void onEnabled(final ConfigurationContext context) {
prettyPrint = context.getProperty(PRETTY_PRINT_JSON).asBoolean();
}
@Override
public RecordSetWriter createWriter(final ComponentLog logger) {
return new WriteJsonResult(logger, prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat());
}
}

Просмотреть файл

@ -0,0 +1,56 @@
/*
* 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.json;
import java.io.IOException;
import java.io.InputStream;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RowRecordReaderFactory;
import org.apache.nifi.serialization.UserTypeOverrideRowReader;
@Tags({"json", "tree", "record", "reader", "parser"})
@CapabilityDescription("Parses JSON into individual Record objects. The Record that is produced will contain all top-level "
+ "elements of the corresponding JSON Object. If the JSON has nested arrays, those values will be represented as an Object array for that field. "
+ "Nested JSON objects will be represented as a Map. "
+ "The root JSON element can be either a single element or an array of JSON elements, and each "
+ "element in that array will be treated as a separate record. If any of the elements has a nested array or a nested "
+ "element, they will be returned as OBJECT or ARRAY types (respectively), not flattened out into individual fields. "
+ "The schema for the record is determined by the first JSON element in the array, if the incoming FlowFile is a JSON array. "
+ "This means that if a field does not exist in the first JSON object, then it will be skipped in all subsequent JSON objects. "
+ "The data type of a field can be overridden by adding a property to "
+ "the controller service where the name of the property matches the JSON field name and the value of the property is "
+ "the data type to use. If that field does not exist in a JSON element, the field will be assumed to be null. "
+ "See the Usage of the Controller Service for more information.")
@SeeAlso(JsonPathReader.class)
@DynamicProperty(name = "<name of JSON field>", value = "<data type of JSON field>",
description = "User-defined properties are used to indicate that the values of a specific field should be interpreted as a "
+ "user-defined data type (e.g., int, double, float, date, etc.)", supportsExpressionLanguage = false)
public class JsonTreeReader extends UserTypeOverrideRowReader implements RowRecordReaderFactory {
@Override
public RecordReader createRecordReader(final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException {
return new JsonTreeRowRecordReader(in, logger, getFieldTypeOverrides());
}
}

Просмотреть файл

@ -0,0 +1,115 @@
/*
* 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.json;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordField;
import org.apache.nifi.serialization.record.RecordSchema;
import org.codehaus.jackson.JsonNode;
public class JsonTreeRowRecordReader extends AbstractJsonRowRecordReader {
private final Map<String, DataType> fieldTypeOverrides;
private RecordSchema schema;
public JsonTreeRowRecordReader(final InputStream in, final ComponentLog logger, final Map<String, DataType> fieldTypeOverrides) throws IOException, MalformedRecordException {
super(in, logger);
this.fieldTypeOverrides = fieldTypeOverrides;
}
@Override
protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException, MalformedRecordException {
if (jsonNode == null) {
return null;
}
final Map<String, Object> values = new HashMap<>(schema.getFieldCount());
for (int i = 0; i < schema.getFieldCount(); i++) {
final RecordField field = schema.getField(i);
final String fieldName = field.getFieldName();
final JsonNode fieldNode = jsonNode.get(fieldName);
final DataType desiredType = field.getDataType();
final Object value = convertField(fieldNode, fieldName, desiredType);
values.put(fieldName, value);
}
return new MapRecord(schema, values);
}
@Override
public RecordSchema getSchema() {
if (schema != null) {
return schema;
}
final List<RecordField> recordFields = new ArrayList<>();
final Optional<JsonNode> firstNodeOption = getFirstJsonNode();
if (firstNodeOption.isPresent()) {
final Iterator<Map.Entry<String, JsonNode>> itr = firstNodeOption.get().getFields();
while (itr.hasNext()) {
final Map.Entry<String, JsonNode> entry = itr.next();
final String elementName = entry.getKey();
final JsonNode node = entry.getValue();
DataType dataType;
final DataType overriddenDataType = fieldTypeOverrides.get(elementName);
if (overriddenDataType == null) {
dataType = determineFieldType(node);
} else {
dataType = overriddenDataType;
}
recordFields.add(new RecordField(elementName, dataType));
}
}
// If there are any overridden field types that we didn't find, add as the last fields.
final Set<String> knownFieldNames = recordFields.stream()
.map(f -> f.getFieldName())
.collect(Collectors.toSet());
for (final Map.Entry<String, DataType> entry : fieldTypeOverrides.entrySet()) {
if (!knownFieldNames.contains(entry.getKey())) {
recordFields.add(new RecordField(entry.getKey(), entry.getValue()));
}
}
schema = new SimpleRecordSchema(recordFields);
return schema;
}
}

Просмотреть файл

@ -0,0 +1,88 @@
/*
* 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.json;
import java.util.Optional;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.RecordFieldType;
public class PropertyNameUtil {
public static String getFieldName(final String propertyName) {
final int colonIndex = propertyName.indexOf(":");
if (colonIndex > -1 && colonIndex < propertyName.length() - 1) {
return propertyName.substring(0, colonIndex);
}
return propertyName;
}
public static boolean hasFieldType(final String propertyName) {
final int colonIndex = propertyName.indexOf(":");
return (colonIndex > -1 && colonIndex < propertyName.length() - 1);
}
public static Optional<String> getFieldTypeName(final String propertyName) {
if (hasFieldType(propertyName)) {
final String[] splits = propertyName.split("\\:");
if (splits.length > 1) {
return Optional.of(splits[1]);
}
return Optional.empty();
}
return Optional.empty();
}
public static Optional<String> getFieldFormat(final String propertyName) {
final String[] splits = propertyName.split("\\:");
if (splits.length != 3) {
return Optional.empty();
}
return Optional.of(splits[2]);
}
public static boolean isFieldTypeValid(final String propertyName) {
final Optional<String> fieldType = getFieldTypeName(propertyName);
if (!fieldType.isPresent()) {
return false;
}
final String typeName = fieldType.get();
final RecordFieldType recordFieldType = RecordFieldType.of(typeName);
return recordFieldType != null;
}
public static Optional<DataType> getDataType(final String propertyName) {
if (isFieldTypeValid(propertyName)) {
final String typeName = getFieldTypeName(propertyName).get();
final RecordFieldType fieldType = RecordFieldType.of(typeName);
final Optional<String> format = getFieldFormat(propertyName);
if (format.isPresent()) {
return Optional.of(fieldType.getDataType(format.get()));
} else {
return Optional.of(fieldType.getDataType());
}
}
return Optional.empty();
}
}

Просмотреть файл

@ -0,0 +1,309 @@
/*
* 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.json;
import java.io.IOException;
import java.io.OutputStream;
import java.math.BigInteger;
import java.sql.Array;
import java.sql.SQLException;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Map;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.DataTypeUtils;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.WriteResult;
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.RecordSchema;
import org.apache.nifi.serialization.record.RecordSet;
import org.apache.nifi.stream.io.NonCloseableOutputStream;
import org.codehaus.jackson.JsonFactory;
import org.codehaus.jackson.JsonGenerationException;
import org.codehaus.jackson.JsonGenerator;
public class WriteJsonResult implements RecordSetWriter {
private final boolean prettyPrint;
private final ComponentLog logger;
private final JsonFactory factory = new JsonFactory();
private final DateFormat dateFormat;
private final DateFormat timeFormat;
private final DateFormat timestampFormat;
public WriteJsonResult(final ComponentLog logger, final boolean prettyPrint, final String dateFormat, final String timeFormat, final String timestampFormat) {
this.prettyPrint = prettyPrint;
this.dateFormat = new SimpleDateFormat(dateFormat);
this.timeFormat = new SimpleDateFormat(timeFormat);
this.timestampFormat = new SimpleDateFormat(timestampFormat);
this.logger = logger;
}
@Override
public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException {
int count = 0;
try (final JsonGenerator generator = factory.createJsonGenerator(new NonCloseableOutputStream(rawOut))) {
if (prettyPrint) {
generator.useDefaultPrettyPrinter();
}
generator.writeStartArray();
Record record;
while ((record = rs.next()) != null) {
count++;
writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
}
generator.writeEndArray();
} catch (final SQLException e) {
throw new IOException("Failed to serialize Result Set to stream", e);
}
return WriteResult.of(count, Collections.emptyMap());
}
@Override
public WriteResult write(final Record record, final OutputStream rawOut) throws IOException {
try (final JsonGenerator generator = factory.createJsonGenerator(new NonCloseableOutputStream(rawOut))) {
if (prettyPrint) {
generator.useDefaultPrettyPrinter();
}
writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject());
} catch (final SQLException e) {
throw new IOException("Failed to write records to stream", e);
}
return WriteResult.of(1, Collections.emptyMap());
}
private void writeRecord(final Record record, final JsonGenerator generator, final GeneratorTask startTask, final GeneratorTask endTask)
throws JsonGenerationException, IOException, SQLException {
try {
final RecordSchema schema = record.getSchema();
startTask.apply(generator);
for (int i = 0; i < schema.getFieldCount(); i++) {
final String fieldName = schema.getField(i).getFieldName();
final Object value = record.getValue(fieldName);
if (value == null) {
generator.writeNullField(fieldName);
continue;
}
generator.writeFieldName(fieldName);
final DataType dataType = schema.getDataType(fieldName).get();
writeValue(generator, value, dataType, i < schema.getFieldCount() - 1);
}
endTask.apply(generator);
} catch (final Exception e) {
logger.error("Failed to write {} with schema {} as a JSON Object due to {}", new Object[] {record, record.getSchema(), e.toString(), e});
throw e;
}
}
private String createDate(final Object value, final DateFormat format) {
if (value == null) {
return null;
}
if (value instanceof Date) {
return format.format((Date) value);
}
if (value instanceof java.sql.Date) {
return format.format(new Date(((java.sql.Date) value).getTime()));
}
if (value instanceof java.sql.Time) {
return format.format(new Date(((java.sql.Time) value).getTime()));
}
if (value instanceof java.sql.Timestamp) {
return format.format(new Date(((java.sql.Timestamp) value).getTime()));
}
return null;
}
private void writeValue(final JsonGenerator generator, final Object value, final DataType dataType, final boolean moreCols)
throws JsonGenerationException, IOException, SQLException {
if (value == null) {
generator.writeNull();
return;
}
final DataType resolvedDataType;
if (dataType.getFieldType() == RecordFieldType.CHOICE) {
resolvedDataType = DataTypeUtils.inferDataType(value);
} else {
resolvedDataType = dataType;
}
switch (resolvedDataType.getFieldType()) {
case DATE:
generator.writeString(createDate(value, dateFormat));
break;
case TIME:
generator.writeString(createDate(value, timeFormat));
break;
case TIMESTAMP:
generator.writeString(createDate(value, timestampFormat));
break;
case DOUBLE:
generator.writeNumber(DataTypeUtils.toDouble(value, 0D));
break;
case FLOAT:
generator.writeNumber(DataTypeUtils.toFloat(value, 0F));
break;
case LONG:
generator.writeNumber(DataTypeUtils.toLong(value, 0L));
break;
case INT:
case BYTE:
case SHORT:
generator.writeNumber(DataTypeUtils.toInteger(value, 0));
break;
case CHAR:
case STRING:
generator.writeString(value.toString());
break;
case BIGINT:
if (value instanceof Long) {
generator.writeNumber(((Long) value).longValue());
} else {
generator.writeNumber((BigInteger) value);
}
break;
case BOOLEAN:
final String stringValue = value.toString();
if ("true".equalsIgnoreCase(stringValue)) {
generator.writeBoolean(true);
} else if ("false".equalsIgnoreCase(stringValue)) {
generator.writeBoolean(false);
} else {
generator.writeString(stringValue);
}
break;
case RECORD: {
final Record record = (Record) value;
writeRecord(record, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject());
break;
}
case ARRAY:
default:
if ("null".equals(value.toString())) {
generator.writeNull();
} else if (value instanceof Map) {
final Map<?, ?> map = (Map<?, ?>) value;
generator.writeStartObject();
int i = 0;
for (final Map.Entry<?, ?> entry : map.entrySet()) {
generator.writeFieldName(entry.getKey().toString());
final boolean moreEntries = ++i < map.size();
writeValue(generator, entry.getValue(), getColType(entry.getValue()), moreEntries);
}
generator.writeEndObject();
} else if (value instanceof List) {
final List<?> list = (List<?>) value;
writeArray(list.toArray(), generator);
} else if (value instanceof Array) {
final Array array = (Array) value;
final Object[] values = (Object[]) array.getArray();
writeArray(values, generator);
} else if (value instanceof Object[]) {
final Object[] values = (Object[]) value;
writeArray(values, generator);
} else {
generator.writeString(value.toString());
}
break;
}
}
private void writeArray(final Object[] values, final JsonGenerator generator) throws JsonGenerationException, IOException, SQLException {
generator.writeStartArray();
for (int i = 0; i < values.length; i++) {
final boolean moreEntries = i < values.length - 1;
final Object element = values[i];
writeValue(generator, element, getColType(element), moreEntries);
}
generator.writeEndArray();
}
private DataType getColType(final Object value) {
if (value instanceof String) {
return RecordFieldType.STRING.getDataType();
}
if (value instanceof Double) {
return RecordFieldType.DOUBLE.getDataType();
}
if (value instanceof Float) {
return RecordFieldType.FLOAT.getDataType();
}
if (value instanceof Integer) {
return RecordFieldType.INT.getDataType();
}
if (value instanceof Long) {
return RecordFieldType.LONG.getDataType();
}
if (value instanceof BigInteger) {
return RecordFieldType.BIGINT.getDataType();
}
if (value instanceof Boolean) {
return RecordFieldType.BOOLEAN.getDataType();
}
if (value instanceof Byte || value instanceof Short) {
return RecordFieldType.INT.getDataType();
}
if (value instanceof Character) {
return RecordFieldType.STRING.getDataType();
}
if (value instanceof java.util.Date || value instanceof java.sql.Date) {
return RecordFieldType.DATE.getDataType();
}
if (value instanceof java.sql.Time) {
return RecordFieldType.TIME.getDataType();
}
if (value instanceof java.sql.Timestamp) {
return RecordFieldType.TIMESTAMP.getDataType();
}
if (value instanceof Object[] || value instanceof List || value instanceof Array) {
return RecordFieldType.ARRAY.getDataType();
}
return RecordFieldType.RECORD.getDataType();
}
@Override
public String getMimeType() {
return "application/json";
}
private static interface GeneratorTask {
void apply(JsonGenerator generator) throws JsonGenerationException, IOException;
}
}

Просмотреть файл

@ -0,0 +1,84 @@
/*
* 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;
import java.util.Arrays;
import java.util.List;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.serialization.record.RecordFieldType;
public abstract class AbstractRecordSetWriter extends AbstractControllerService {
static final PropertyDescriptor DATE_FORMAT = new PropertyDescriptor.Builder()
.name("Date Format")
.description("Specifies the format to use when writing out Date fields")
.expressionLanguageSupported(false)
.defaultValue(RecordFieldType.DATE.getDefaultFormat())
.addValidator(new SimpleDateFormatValidator())
.required(true)
.build();
static final PropertyDescriptor TIME_FORMAT = new PropertyDescriptor.Builder()
.name("Time Format")
.description("Specifies the format to use when writing out Time fields")
.expressionLanguageSupported(false)
.defaultValue(RecordFieldType.TIME.getDefaultFormat())
.addValidator(new SimpleDateFormatValidator())
.required(true)
.build();
static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
.name("Timestamp Format")
.description("Specifies the format to use when writing out Timestamp (date/time) fields")
.expressionLanguageSupported(false)
.defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
.addValidator(new SimpleDateFormatValidator())
.required(true)
.build();
private volatile String dateFormat;
private volatile String timeFormat;
private volatile String timestampFormat;
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return Arrays.asList(DATE_FORMAT, TIME_FORMAT, TIMESTAMP_FORMAT);
}
@OnEnabled
public void captureValues(final ConfigurationContext context) {
this.dateFormat = context.getProperty(DATE_FORMAT).getValue();
this.timeFormat = context.getProperty(TIME_FORMAT).getValue();
this.timestampFormat = context.getProperty(TIMESTAMP_FORMAT).getValue();
}
protected String getDateFormat() {
return dateFormat;
}
protected String getTimeFormat() {
return timeFormat;
}
protected String getTimestampFormat() {
return timestampFormat;
}
}

Просмотреть файл

@ -0,0 +1,165 @@
/*
* 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;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.RecordField;
import org.apache.nifi.serialization.record.RecordFieldType;
import org.apache.nifi.serialization.record.RecordSchema;
public class DataTypeUtils {
public static Double toDouble(final Object value, final Double defaultValue) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).doubleValue();
}
if (value instanceof String) {
return Double.parseDouble((String) value);
}
return defaultValue;
}
public static Float toFloat(final Object value, final Float defaultValue) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).floatValue();
}
if (value instanceof String) {
return Float.parseFloat((String) value);
}
return defaultValue;
}
public static Long toLong(final Object value, final Long defaultValue) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).longValue();
}
if (value instanceof String) {
return Long.parseLong((String) value);
}
return defaultValue;
}
public static Integer toInteger(final Object value, final Integer defaultValue) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).intValue();
}
if (value instanceof String) {
return Integer.parseInt((String) value);
}
return defaultValue;
}
/**
* Deduces the type of RecordFieldType that should be used for a value of the given type,
* or returns <code>null</code> if the value is null
*
* @param value the value whose type should be deduced
* @return the type of RecordFieldType that should be used for a value of the given type,
* or <code>null</code> if the value is null
*/
public static DataType inferDataType(final Object value) {
if (value == null) {
return null;
}
if (value instanceof String) {
return RecordFieldType.STRING.getDataType();
}
if (value instanceof Long) {
return RecordFieldType.LONG.getDataType();
}
if (value instanceof Integer) {
return RecordFieldType.INT.getDataType();
}
if (value instanceof Double) {
return RecordFieldType.DOUBLE.getDataType();
}
if (value instanceof Float) {
return RecordFieldType.FLOAT.getDataType();
}
if (value instanceof Boolean) {
return RecordFieldType.BOOLEAN.getDataType();
}
if (value instanceof Byte) {
return RecordFieldType.BYTE.getDataType();
}
if (value instanceof Character) {
return RecordFieldType.CHAR.getDataType();
}
if (value instanceof Short) {
return RecordFieldType.SHORT.getDataType();
}
if (value instanceof Date) {
return RecordFieldType.DATE.getDataType();
}
if (value instanceof Object[] || value instanceof List) {
return RecordFieldType.ARRAY.getDataType();
}
if (value instanceof Map) {
@SuppressWarnings("unchecked")
final Map<String, Object> map = (Map<String, Object>) value;
final RecordSchema childSchema = determineSchema(map);
return RecordFieldType.RECORD.getDataType(childSchema);
}
return RecordFieldType.RECORD.getDataType();
}
public static RecordSchema determineSchema(final Map<String, Object> valueMap) {
final List<RecordField> fields = new ArrayList<>(valueMap.size());
for (final Map.Entry<String, Object> entry : valueMap.entrySet()) {
final DataType valueType = inferDataType(entry.getValue());
final String fieldName = entry.getKey();
final RecordField field = new RecordField(fieldName, valueType);
fields.add(field);
}
return new SimpleRecordSchema(fields);
}
}

Просмотреть файл

@ -0,0 +1,48 @@
/*
* 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;
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 {
@Override
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
try {
new SimpleDateFormat(input);
} catch (final Exception e) {
return new ValidationResult.Builder()
.input(input)
.subject(subject)
.valid(false)
.explanation("Invalid Date format: " + e.getMessage())
.build();
}
return new ValidationResult.Builder()
.input(input)
.subject(subject)
.valid(true)
.build();
}
}

Просмотреть файл

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.serialization;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.serialization.DataTypeValidator;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.RecordFieldType;
public abstract class UserTypeOverrideRowReader extends AbstractControllerService {
private volatile Map<String, DataType> fieldTypeOverrides;
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.dynamic(true)
.addValidator(new DataTypeValidator())
.build();
}
@OnEnabled
public void createFieldTypeOverrides(final ConfigurationContext context) {
final Map<String, DataType> overrides = new HashMap<>(context.getProperties().size());
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
if (!entry.getKey().isDynamic()) {
continue;
}
final String fieldName = entry.getKey().getName();
final String dataTypeName = entry.getValue();
if (dataTypeName == null) {
continue;
}
final DataType dataType;
final String[] splits = dataTypeName.split("\\:");
if (splits.length == 2) {
final RecordFieldType fieldType = RecordFieldType.of(splits[0]);
final String format = splits[1];
dataType = fieldType.getDataType(format);
} else {
final RecordFieldType fieldType = RecordFieldType.of(dataTypeName);
dataType = fieldType.getDataType();
}
overrides.put(fieldName, dataType);
}
this.fieldTypeOverrides = Collections.unmodifiableMap(overrides);
}
protected Map<String, DataType> getFieldTypeOverrides() {
return fieldTypeOverrides;
}
}

Просмотреть файл

@ -0,0 +1,80 @@
/*
* 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.text;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@Tags({"text", "freeform", "expression", "language", "el", "resultset", "writer", "serialize"})
@CapabilityDescription("Writes the contents of a Database ResultSet as free-form text. The configured "
+ "text is able to make use of the Expression Language to reference each of the columns that are available "
+ "in the ResultSet. Each record in the ResultSet will be separated by a single newline character.")
public class FreeFormTextRecordSetWriter extends AbstractControllerService implements RecordSetWriterFactory {
static final PropertyDescriptor TEXT = new PropertyDescriptor.Builder()
.name("Text")
.description("The text to use when writing the results. This property will evaluate the Expression Language using any of the columns available to the Result Set. For example, if the "
+ "following SQL Query is used: \"SELECT Name, COUNT(*) AS Count\" then the Expression can reference \"Name\" and \"Count\", such as \"${Name:toUpper()} ${Count:minus(1)}\"")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.required(true)
.build();
static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
.name("Character Set")
.description("The Character set to use when writing the data to the FlowFile")
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.defaultValue("UTF-8")
.expressionLanguageSupported(false)
.required(true)
.build();
private volatile PropertyValue textValue;
private volatile Charset characterSet;
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(TEXT);
properties.add(CHARACTER_SET);
return properties;
}
@OnEnabled
public void onEnabled(final ConfigurationContext context) {
textValue = context.getProperty(TEXT);
characterSet = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
}
@Override
public RecordSetWriter createWriter(final ComponentLog logger) {
return new FreeFormTextWriter(textValue, characterSet);
}
}

Просмотреть файл

@ -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.text;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.charset.Charset;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.Record;
import org.apache.nifi.serialization.record.RecordSchema;
import org.apache.nifi.serialization.record.RecordSet;
public class FreeFormTextWriter implements RecordSetWriter {
private static final byte NEW_LINE = (byte) '\n';
private final PropertyValue propertyValue;
private final Charset charset;
public FreeFormTextWriter(final PropertyValue textPropertyValue, final Charset characterSet) {
propertyValue = textPropertyValue;
charset = characterSet;
}
@Override
public WriteResult write(final RecordSet recordSet, final OutputStream out) throws IOException {
int count = 0;
try {
final RecordSchema schema = recordSet.getSchema();
final String[] colNames = getColumnNames(schema);
Record record;
while ((record = recordSet.next()) != null) {
count++;
write(record, out, colNames);
}
} catch (final Exception e) {
throw new ProcessException(e);
}
return WriteResult.of(count, Collections.emptyMap());
}
private String[] getColumnNames(final RecordSchema schema) {
final int numCols = schema.getFieldCount();
final String[] columnNames = new String[numCols];
for (int i = 0; i < numCols; i++) {
columnNames[i] = schema.getField(i).getFieldName();
}
return columnNames;
}
@Override
public WriteResult write(final Record record, final OutputStream out) throws IOException {
write(record, out, getColumnNames(record.getSchema()));
return WriteResult.of(1, Collections.emptyMap());
}
private void write(final Record record, final OutputStream out, final String[] columnNames) throws IOException {
final int numCols = columnNames.length;
final Map<String, String> values = new HashMap<>(numCols);
for (int i = 0; i < numCols; i++) {
final String columnName = columnNames[i];
final String columnValue = record.getAsString(columnName);
values.put(columnName, columnValue);
}
final String evaluated = propertyValue.evaluateAttributeExpressions(values).getValue();
out.write(evaluated.getBytes(charset));
out.write(NEW_LINE);
}
@Override
public String getMimeType() {
return "text/plain";
}
}

Просмотреть файл

@ -0,0 +1,28 @@
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.nifi.avro.AvroReader
org.apache.nifi.avro.AvroRecordSetWriter
org.apache.nifi.json.JsonTreeReader
org.apache.nifi.json.JsonPathReader
org.apache.nifi.json.JsonRecordSetWriter
org.apache.nifi.csv.CSVReader
org.apache.nifi.csv.CSVRecordSetWriter
org.apache.nifi.grok.GrokReader
org.apache.nifi.text.FreeFormTextRecordSetWriter

Просмотреть файл

@ -0,0 +1,115 @@
# 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.
# Log Levels
LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)|FINE|FINER|FINEST|CONFIG
# Syslog Dates: Month Day HH:MM:SS
SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
PROG (?:[\w._/%-]+)
SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
SYSLOGHOST %{IPORHOST}
SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
# Months: January, Feb, 3, 03, 12, December
MONTH \b(?:Jan(?:uary)?|Feb(?:ruary)?|Mar(?:ch)?|Apr(?:il)?|May|Jun(?:e)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|Oct(?:ober)?|Nov(?:ember)?|Dec(?:ember)?)\b
MONTHNUM (?:0?[1-9]|1[0-2])
MONTHNUM2 (?:0[1-9]|1[0-2])
MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
# Days: Monday, Tue, Thu, etc...
DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
# Years?
YEAR (?>\d\d){1,2}
HOUR (?:2[0123]|[01]?[0-9])
MINUTE (?:[0-5][0-9])
# '60' is a leap second in most time standards and thus is valid.
SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
DATE_US_MONTH_DAY_YEAR %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
DATE_US_YEAR_MONTH_DAY %{YEAR}[/-]%{MONTHNUM}[/-]%{MONTHDAY}
DATE_US %{DATE_US_MONTH_DAY_YEAR}|%{DATE_US_YEAR_MONTH_DAY}
DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
ISO8601_SECOND (?:%{SECOND}|60)
TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
DATE %{DATE_US}|%{DATE_EU}
DATESTAMP %{DATE}[- ]%{TIME}
TZ (?:[PMCE][SD]T|UTC)
DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
POSINT \b(?:[1-9][0-9]*)\b
NONNEGINT \b(?:[0-9]+)\b
WORD \b\w+\b
NOTSPACE \S+
SPACE \s*
DATA .*?
GREEDYDATA .*
QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
USERNAME [a-zA-Z0-9._-]+
USER %{USERNAME}
INT (?:[+-]?(?:[0-9]+))
BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
NUMBER (?:%{BASE10NUM})
BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
# Networking
MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
IPV4 (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
IP (?:%{IPV6}|%{IPV4})
HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
HOST %{HOSTNAME}
IPORHOST (?:%{HOSTNAME}|%{IP})
HOSTPORT %{IPORHOST}:%{POSINT}
# paths
PATH (?:%{UNIXPATH}|%{WINPATH})
UNIXPATH (?>/(?>[\w_%!$@:.,-]+|\\.)*)+
TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
URIPROTO [A-Za-z]+(\+[A-Za-z+]+)?
URIHOST %{IPORHOST}(?::%{POSINT:port})?
# uripath comes loosely from RFC1738, but mostly from what Firefox
# doesn't turn into %XX
URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+
#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]]*
URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
# Shortcuts
QS %{QUOTEDSTRING}
# Log formats
SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
COMMONAPACHELOG %{IPORHOST:clientip} %{USER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-)
COMBINEDAPACHELOG %{COMMONAPACHELOG} %{QS:referrer} %{QS:agent}

Просмотреть файл

@ -0,0 +1,185 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8"/>
<title>CSVReader</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
</head>
<body>
<p>
The CSVReader Controller Service, expects input in such a way that the first line of a FlowFile specifies the name of
each column in the data. Following the first line, the rest of the FlowFile is expected to be valid CSV data from which
to form appropriate Records. By default, the schema for a FlowFile is inferred by extracting the name of each column from
the first line of the CSV and assumes that all columns are of type <code>string</code>. Of course, we may want to treat some
columns as a data type other than <code>string</code>. This can be accomplished by adding a user-defined property where the
name of the property is the same as the name of a CSV column and the value of the property is the data type to use.
</p>
<p>
When specifying a data type for a field, the following values are valid:
</p>
<ul>
<li><b>string</b></li>
<li><b>boolean</b></li>
<li><b>byte</b></li>
<li><b>char</b></li>
<li><b>short</b></li>
<li><b>int</b></li>
<li><b>bigint</b></li>
<li><b>long</b></li>
<li><b>float</b></li>
<li><b>double</b></li>
<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).</li>
<li><b>object</b> - <i>This data type does not apply to CSV data.</i></li>
<li><b>array</b> - <i>This data type does not apply to CSV data.</i></li>
</ul>
<p>
As an example, consider a FlowFile whose contents consists of the following:
</p>
<code>
id, name, balance, notes<br />
1, John, 48.23, "Our very<br />
first customer!"<br />
2, Jane, 1245.89,<br />
3, Frank Franklin, "48481.29",<br />
</code>
<p>
Additionally, let's consider that this Controller Service is configured with the following user-defined properties:
</p>
<table>
<head>
<th>Property Name</th>
<th>Property Value</th>
</head>
<body>
<tr>
<td>balance</td>
<td><code>float</code></td>
</tr>
</body>
</table>
<p>
In this case, the result will be that this FlowFile consists of 3 different records. The first record will contain the following values:
</p>
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>id</td>
<td>1</td>
</tr>
<tr>
<td>name</td>
<td>John</td>
</tr>
<tr>
<td>balance</td>
<td>48.23</td>
</tr>
<tr>
<td>notes</td>
<td>Our very<br />first customer!</td>
</tr>
</body>
</table>
<p>
The second record will contain the following values:
</p>
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>id</td>
<td>2</td>
</tr>
<tr>
<td>name</td>
<td>Jane</td>
</tr>
<tr>
<td>balance</td>
<td>1245.89</td>
</tr>
<tr>
<td>notes</td>
<td></td>
</tr>
</body>
</table>
<p>
The third record will contain the following values:
</p>
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>id</td>
<td>3</td>
</tr>
<tr>
<td>name</td>
<td>Frank Franklin</td>
</tr>
<tr>
<td>balance</td>
<td>48481.29</td>
</tr>
<tr>
<td>notes</td>
<td></td>
</tr>
</body>
</table>
</body>
</html>

Просмотреть файл

@ -0,0 +1,396 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8"/>
<title>GrokReader</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
</head>
<body>
<p>
The GrokReader Controller Service, provides a means for parsing and structuring input that is
made up of unstructured text, such as log files. Grok allows users to add a naming construct to
Regular Expressions such that they can be composed in order to create expressions that are easier
to manage and work with. This Controller Service consists of one Required Property and one Optional
Property. The Optional Property is named <code>Grok Pattern File</code> and specifies the filename of
a file that contains Grok Patterns that can be used for parsing log data. If not specified, a default
patterns file will be used. Its contains are provided below.
</p>
<p>
The Required Property is named <code>Grok Expression</code> and specifies how to parse each
incoming record. This is done by providing a Grok Expression such as:
<code>%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \[%{DATA:thread}\] %{DATA:class} %{GREEDYDATA:message}</code>.
This Expression will parse Apache NiFi log messages. This is accomplished by specifying that a line begins
with the <code>TIMESTAMP_ISO8601</code> pattern (which is a Regular Expression defined in the default
Grok Patterns File). The value that matches this pattern is then given the name <code>timestamp</code>. As a result,
the value that matches this pattern will be assigned to a field named <code>timestamp</code> in the Record that
produced by this Controller Service.
</p>
<p>
If a line is encountered in the FlowFile that does not match the configured Grok Expression, it is assumed that the line
is part of the previous message. If the line is the start of a stack trace, then the entire stack trace is read in and assigned
to a field named <code>STACK_TRACE</code>. Otherwise, the line is appended to the last field defined in the Grok Expression. This
is done because typically the last field is a 'message' type of field, which can consist of new-lines.
</p>
<p>
By default, all fields that are extracted are considered to be of type <code>string</code>. This can be overridden
by adding a user-defined property where the name of the property matches the name of the field that is present in the
configured Grok Expression. The value of the user-defined property is the data type to use.
When specifying a data type for a field, the following values are valid:
</p>
<ul>
<li><b>string</b></li>
<li><b>boolean</b></li>
<li><b>byte</b></li>
<li><b>char</b></li>
<li><b>short</b></li>
<li><b>int</b></li>
<li><b>bigint</b></li>
<li><b>long</b></li>
<li><b>float</b></li>
<li><b>double</b></li>
<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).</li>
<li><b>object</b> - <i>This data type does not apply to CSV data.</i></li>
<li><b>array</b> - <i>This data type does not apply to CSV data.</i></li>
</ul>
<h2>
Examples
</h2>
<p>
As an example, consider that this Controller Service is configured with the following properties:
</p>
<table>
<head>
<th>Property Name</th>
<th>Property Value</th>
</head>
<body>
<tr>
<td>Grok Expression</td>
<td><code>%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \[%{DATA:thread}\] %{DATA:class} %{GREEDYDATA:message}</code></td>
</tr>
</body>
</table>
<p>
Additionally, let's consider a FlowFile whose contents consists of the following:
</p>
<code><pre>
2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'
2016-08-04 13:26:32,474 ERROR [Leader Election Notification Thread-2] o.apache.nifi.controller.FlowController One
Two
Three
org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
... 12 common frames omitted
2016-08-04 13:26:35,475 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
</pre></code>
<p>
In this case, the result will be that this FlowFile consists of 3 different records. The first record will contain the following values:
</p>
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>timestamp</td>
<td>2016-08-04 13:26:32,473</td>
</tr>
<tr>
<td>level</td>
<td>INFO</td>
</tr>
<tr>
<td>thread</td>
<td>Leader Election Notification Thread-1</td>
</tr>
<tr>
<td>class</td>
<td>o.a.n.c.l.e.CuratorLeaderElectionManager</td>
</tr>
<tr>
<td>message</td>
<td>org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'</td>
</tr>
<tr>
<td>STACK_TRACE</td>
<td><i>null</i></td>
</tr>
</body>
</table>
<p>
The second record will contain the following values:
</p>
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>timestamp</td>
<td>2016-08-04 13:26:32,474</td>
</tr>
<tr>
<td>level</td>
<td>ERROR</td>
</tr>
<tr>
<td>thread</td>
<td>Leader Election Notification Thread-2</td>
</tr>
<tr>
<td>class</td>
<td>o.apache.nifi.controller.FlowController</td>
</tr>
<tr>
<td>message</td>
<td>One<br />
Two<br />
Three</td>
</tr>
<tr>
<td>STACK_TRACE</td>
<td>
<pre>
org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185)
... 12 common frames omitted
</pre></td>
</tr>
</body>
</table>
<p>
The third record will contain the following values:
</p>
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>timestamp</td>
<td>2016-08-04 13:26:35,475</td>
</tr>
<tr>
<td>level</td>
<td>WARN</td>
</tr>
<tr>
<td>thread</td>
<td>Curator-Framework-0</td>
</tr>
<tr>
<td>class</td>
<td>org.apache.curator.ConnectionState</td>
</tr>
<tr>
<td>message</td>
<td>Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.</td>
</tr>
<tr>
<td>STACK_TRACE</td>
<td><i>null</i></td>
</tr>
</body>
</table>
<h2>
</h2>
<h2>Default Patterns</h2>
<p>
The following patterns are available in the default Grok Pattern File:
</p>
<code>
<pre>
# Log Levels
LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)|FINE|FINER|FINEST|CONFIG
# Syslog Dates: Month Day HH:MM:SS
SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
PROG (?:[\w._/%-]+)
SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
SYSLOGHOST %{IPORHOST}
SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
# Months: January, Feb, 3, 03, 12, December
MONTH \b(?:Jan(?:uary)?|Feb(?:ruary)?|Mar(?:ch)?|Apr(?:il)?|May|Jun(?:e)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|Oct(?:ober)?|Nov(?:ember)?|Dec(?:ember)?)\b
MONTHNUM (?:0?[1-9]|1[0-2])
MONTHNUM2 (?:0[1-9]|1[0-2])
MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
# Days: Monday, Tue, Thu, etc...
DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
# Years?
YEAR (?>\d\d){1,2}
HOUR (?:2[0123]|[01]?[0-9])
MINUTE (?:[0-5][0-9])
# '60' is a leap second in most time standards and thus is valid.
SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
DATE_US_MONTH_DAY_YEAR %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
DATE_US_YEAR_MONTH_DAY %{YEAR}[/-]%{MONTHNUM}[/-]%{MONTHDAY}
DATE_US %{DATE_US_MONTH_DAY_YEAR}|%{DATE_US_YEAR_MONTH_DAY}
DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
ISO8601_SECOND (?:%{SECOND}|60)
TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
DATE %{DATE_US}|%{DATE_EU}
DATESTAMP %{DATE}[- ]%{TIME}
TZ (?:[PMCE][SD]T|UTC)
DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
POSINT \b(?:[1-9][0-9]*)\b
NONNEGINT \b(?:[0-9]+)\b
WORD \b\w+\b
NOTSPACE \S+
SPACE \s*
DATA .*?
GREEDYDATA .*
QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
USERNAME [a-zA-Z0-9._-]+
USER %{USERNAME}
INT (?:[+-]?(?:[0-9]+))
BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
NUMBER (?:%{BASE10NUM})
BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
# Networking
MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
IPV4 (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
IP (?:%{IPV6}|%{IPV4})
HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
HOST %{HOSTNAME}
IPORHOST (?:%{HOSTNAME}|%{IP})
HOSTPORT %{IPORHOST}:%{POSINT}
# paths
PATH (?:%{UNIXPATH}|%{WINPATH})
UNIXPATH (?>/(?>[\w_%!$@:.,-]+|\\.)*)+
TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
URIPROTO [A-Za-z]+(\+[A-Za-z+]+)?
URIHOST %{IPORHOST}(?::%{POSINT:port})?
# uripath comes loosely from RFC1738, but mostly from what Firefox
# doesn't turn into %XX
URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+
#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]]*
URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
# Shortcuts
QS %{QUOTEDSTRING}
# Log formats
SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
COMMONAPACHELOG %{IPORHOST:clientip} %{USER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-)
COMBINEDAPACHELOG %{COMMONAPACHELOG} %{QS:referrer} %{QS:agent}
</pre>
</code>
</body>
</html>

Просмотреть файл

@ -0,0 +1,227 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8"/>
<title>JsonPathReader</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
</head>
<body>
<p>
The JsonPathReader Controller Service, parses FlowFiles that are in the JSON format. User-defined properties
specify how to extract all relevant fields from the JSON in order to create a row-oriented record. The Controller
Service will not be valid unless at least one JSON Path is provided. Unlike the
<a href="../org.apache.nifi.json.FlatJsonReader/additionalDetails.html">FlatJsonReader</a> Controller Service, this
service will return a record that contains only those fields that have been configured via JSON Path.
</p>
<p>
If the root of the FlowFile's JSON is a JSON Array, each JSON Object found in that array will be treated as a separate
Record, not as a single record made up of an array. If the root of the FlowFile's JSON is a JSON Object, it will be
evaluated as a single Record.
</p>
<p>
Supplying a JSON Path is accomplished by adding a user-defined property where the name of the property becomes the name
of the field in the Record that is returned. The value of the property must be a valid JSON Path expression. This JSON Path
will be evaluated against each top-level JSON Object in the FlowFile, and the result will be the value of the field whose
name is specified by the property name. By default, the type of each field is inferred automatically based on the values of
the first JSON Object encountered for the FlowFile. This can be overridden by changing the name of the user-defined property
by adding a colon (:) and specifying the data type. For example: <code>balance:double</code> or <code>dob:date:MM/dd/yyyy</code>.
In this case, the data type and option format are not included in the field name. So for the aforementioned examples, we would
end up with field names <code>balance</code> and <code>dob</code>.
</p>
<p>
When specifying a data type for a field, the following values are valid:
</p>
<ul>
<li><b>string</b></li>
<li><b>boolean</b></li>
<li><b>byte</b></li>
<li><b>char</b></li>
<li><b>short</b></li>
<li><b>int</b></li>
<li><b>bigint</b></li>
<li><b>long</b></li>
<li><b>float</b></li>
<li><b>double</b></li>
<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).</li>
<li><b>object</b> - The value will be returned as a <code>Map&lt;String, Object&gt;</code>. The types of the values in the Map
are always inferred. The type used for the values may not be the same for each record. For example, consider the following
JSON array:
<br /><br />
<code>
[{
id: 17,
name: "John",
child: {
id: "1"
},
siblingIds: [4, "8"]
},
<br />{
id: 98,
name: "Jane",
child: {
id: 2
},
siblingIds: []
}]
</code>
<br /><br />
In this case, the <code>child</code> element would be inferred to be of type <code>object</code>. Since nested types
are inferred on a per-record basis, for the first record, the <code>child</code> field would return a <code>Map</code>
where the value of the <code>id</code> entry is a <code>string</code>. However, for the second record, the <code>child</code>
field would return a <code>Map</code> where the value of the <code>id</code> entry is an <code>int</code>.
<br />
Moreover, the <code>siblingIds</code> of the John will be an <code>array</code> where the first element is an <code>int</code>
and the second element is a <code>string</code>. The <code>siblingIds</code> of Jane will be an empty array.
</li>
<li><b>array</b> - An array of values. The types of the values are always inferred and may not be the same for each element
in the array, or for two arrays from different JSON objects.</li>
</ul>
<p>
As an example, consider a FlowFile whose content contains the following JSON:
</p>
<code>
[{
id: 17,
name: "John",
child: {
id: "1"
},
siblingIds: [4, "8"]
},
<br />{
id: 98,
name: "Jane",
child: {
id: 2
},
gender: "F",
siblingIds: []
}]
</code>
<p>
If we configure this Controller Service with the following user-defined properties:
<table>
<head>
<th>Property Name</th>
<th>Property Value</th>
</head>
<body>
<tr>
<td>id</td>
<td><code>$.id</code></td>
</tr>
<tr>
<td>name</td>
<td><code>$.name</code></td>
</tr>
<tr>
<td>childId:long</td>
<td><code>$.child.id</code></td>
</tr>
<tr>
<td>gender:string</td>
<td><code>$.gender</code></td>
</tr>
</body>
</table>
</p>
<p>
In this case, the FlowFile will generate two Records. The first record will consist of the following key/value pairs:
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>id</td>
<td>17</td>
</tr>
<tr>
<td>name</td>
<td>John</td>
</tr>
<tr>
<td>childId</td>
<td>1</td>
</tr>
<tr>
<td>gender</td>
<td><i>null</i></td>
</tr>
</body>
</table>
</p>
<p>
The second record will consist of the following key/value pairs:
<table>
<head>
<th>Field Name</th>
<th>Field Value</th>
</head>
<body>
<tr>
<td>id</td>
<td>98</td>
</tr>
<tr>
<td>name</td>
<td>Jane</td>
</tr>
<tr>
<td>childId</td>
<td>2</td>
</tr>
<tr>
<td>gender</td>
<td>F</td>
</tr>
</body>
</table>
</p>
</body>
</html>

Просмотреть файл

@ -0,0 +1,102 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8"/>
<title>JsonTreeReader</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
</head>
<body>
<p>
The JsonTreeReader Controller Service, by default, derives the schema for a FlowFile
based on the first JSON Object in the FlowFile. For each field found, the data type
is inferred. However, the type of a field can be overridden by adding a user-defined property to
the Controller Service. The name of the property should be the same as the name of the
JSON field. The value of the property denotes the data type of the corresponding field.
If no JSON field is found with a matching name, then a field will be added to the schema,
and a <code>null</code> value will be used for any record for which the JSON field
is not present. If a field is found with a matching name, but the type is different,
the Controller Service will attempt to coerce the value into the user-defined type. If unable
to do so, an Exception will be thrown.
</p>
<p>
When specifying a data type for a field, the following values are valid:
</p>
<ul>
<li><b>string</b></li>
<li><b>boolean</b></li>
<li><b>byte</b></li>
<li><b>char</b></li>
<li><b>short</b></li>
<li><b>int</b></li>
<li><b>bigint</b></li>
<li><b>long</b></li>
<li><b>float</b></li>
<li><b>double</b></li>
<li><b>date</b> - A date with no time field. By default, the format used is <code>yyyy-MM-dd</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>date:MM/dd/yyyy</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>time</b> - A time with no date field. By default, the format used is <code>HH:mm:ss</code>. This can be overridden
by adding a colon (:) followed by the desired format. For example: <code>time:hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).
</li>
<li><b>timestamp</b> - A field that represents both a date and time. By default, the format used is
<code>yyyy-MM-dd HH:mm:ss</code>. This can be overridden by adding a colon (:) followed by the desired format. For example:
<code>MM/dd/yyyy hh:mm:ss a</code>. The format to use is
that of Java's SimpleDateFormat (see <a href="http://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html">
SimpleDateFormat Patterns</a> for more information).</li>
<li><b>object</b> - The value will be returned as a <code>Map&lt;String, Object&gt;</code>. The types of the values in the Map
are always inferred. The type used for the values may not be the same for each record. For example, consider the following
JSON array:
<br /><br />
<code>
[{
id: 17,
name: "John",
child: {
id: "1"
},
siblingIds: [4, "8"]
},
<br />{
id: 98,
name: "Jane",
child: {
id: 2
},
siblingIds: []
}]
</code>
<br /><br />
In this case, the <code>child</code> element would be inferred to be of type <code>object</code>. Since nested types
are inferred on a per-record basis, for the first record, the <code>child</code> field would return a <code>Map</code>
where the value of the <code>id</code> entry is a <code>string</code>. However, for the second record, the <code>child</code>
field would return a <code>Map</code> where the value of the <code>id</code> entry is an <code>int</code>.
<br />
Moreover, the <code>siblingIds</code> of the John will be an <code>array</code> where the first element is an <code>int</code>
and the second element is a <code>string</code>. The <code>siblingIds</code> of Jane will be an empty array.
</li>
<li><b>array</b> - An array of values. The types of the values are always inferred and may not be the same for each element
in the array, or for two arrays from different JSON objects.</li>
</ul>
</body>
</html>

Просмотреть файл

@ -0,0 +1,221 @@
/*
* 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.avro;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.Schema.Type;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.MapRecord;
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.junit.Test;
public class TestAvroRecordReader {
@Test
public void testDataTypes() throws IOException, MalformedRecordException {
final List<Field> accountFields = new ArrayList<>();
accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, null));
accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, null));
final Schema accountSchema = Schema.createRecord("account", null, null, false);
accountSchema.setFields(accountFields);
final List<Field> catFields = new ArrayList<>();
catFields.add(new Field("catTailLength", Schema.create(Type.INT), null, null));
catFields.add(new Field("catName", Schema.create(Type.STRING), null, null));
final Schema catSchema = Schema.createRecord("cat", null, null, false);
catSchema.setFields(catFields);
final List<Field> dogFields = new ArrayList<>();
dogFields.add(new Field("dogTailLength", Schema.create(Type.INT), null, null));
dogFields.add(new Field("dogName", Schema.create(Type.STRING), null, null));
final Schema dogSchema = Schema.createRecord("dog", null, null, false);
dogSchema.setFields(dogFields);
final List<Field> fields = new ArrayList<>();
fields.add(new Field("name", Schema.create(Type.STRING), null, null));
fields.add(new Field("age", Schema.create(Type.INT), null, null));
fields.add(new Field("balance", Schema.create(Type.DOUBLE), null, null));
fields.add(new Field("rate", Schema.create(Type.FLOAT), null, null));
fields.add(new Field("debt", Schema.create(Type.BOOLEAN), null, null));
fields.add(new Field("nickname", Schema.create(Type.NULL), null, null));
fields.add(new Field("binary", Schema.create(Type.BYTES), null, null));
fields.add(new Field("fixed", Schema.createFixed("fixed", null, null, 5), null, null));
fields.add(new Field("map", Schema.createMap(Schema.create(Type.STRING)), null, null));
fields.add(new Field("array", Schema.createArray(Schema.create(Type.LONG)), null, null));
fields.add(new Field("account", accountSchema, null, null));
fields.add(new Field("desiredbalance", Schema.createUnion( // test union of NULL and other type with no value
Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
null, null));
fields.add(new Field("dreambalance", Schema.createUnion( // test union of NULL and other type with a value
Arrays.asList(Schema.create(Type.NULL), Schema.create(Type.DOUBLE))),
null, null));
fields.add(new Field("favAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, null));
fields.add(new Field("otherFavAnimal", Schema.createUnion(Arrays.asList(catSchema, dogSchema)), null, null));
final Schema schema = Schema.createRecord("record", null, null, false);
schema.setFields(fields);
final byte[] source;
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final Map<String, String> map = new HashMap<>();
map.put("greeting", "hello");
map.put("salutation", "good-bye");
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter);
final DataFileWriter<GenericRecord> writer = dataFileWriter.create(schema, baos)) {
final GenericRecord record = new GenericData.Record(schema);
record.put("name", "John");
record.put("age", 33);
record.put("balance", 1234.56D);
record.put("rate", 0.045F);
record.put("debt", false);
record.put("binary", ByteBuffer.wrap("binary".getBytes(StandardCharsets.UTF_8)));
record.put("fixed", new GenericData.Fixed(Schema.create(Type.BYTES), "fixed".getBytes(StandardCharsets.UTF_8)));
record.put("map", map);
record.put("array", Arrays.asList(1L, 2L));
record.put("dreambalance", 10_000_000.00D);
final GenericRecord accountRecord = new GenericData.Record(accountSchema);
accountRecord.put("accountId", 83L);
accountRecord.put("accountName", "Checking");
record.put("account", accountRecord);
final GenericRecord catRecord = new GenericData.Record(catSchema);
catRecord.put("catTailLength", 1);
catRecord.put("catName", "Meow");
record.put("otherFavAnimal", catRecord);
final GenericRecord dogRecord = new GenericData.Record(dogSchema);
dogRecord.put("dogTailLength", 14);
dogRecord.put("dogName", "Fido");
record.put("favAnimal", dogRecord);
writer.append(record);
}
source = baos.toByteArray();
try (final InputStream in = new ByteArrayInputStream(source)) {
final AvroRecordReader reader = new AvroRecordReader(in);
final RecordSchema recordSchema = reader.getSchema();
assertEquals(15, recordSchema.getFieldCount());
assertEquals(RecordFieldType.STRING, recordSchema.getDataType("name").get().getFieldType());
assertEquals(RecordFieldType.INT, recordSchema.getDataType("age").get().getFieldType());
assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("balance").get().getFieldType());
assertEquals(RecordFieldType.FLOAT, recordSchema.getDataType("rate").get().getFieldType());
assertEquals(RecordFieldType.BOOLEAN, recordSchema.getDataType("debt").get().getFieldType());
assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("nickname").get().getFieldType());
assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("binary").get().getFieldType());
assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("fixed").get().getFieldType());
assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("map").get().getFieldType());
assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("array").get().getFieldType());
assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("account").get().getFieldType());
assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("desiredbalance").get().getFieldType());
assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("dreambalance").get().getFieldType());
assertEquals(RecordFieldType.CHOICE, recordSchema.getDataType("favAnimal").get().getFieldType());
assertEquals(RecordFieldType.CHOICE, recordSchema.getDataType("otherFavAnimal").get().getFieldType());
final Object[] values = reader.nextRecord().getValues();
assertEquals(15, values.length);
assertEquals("John", values[0]);
assertEquals(33, values[1]);
assertEquals(1234.56D, values[2]);
assertEquals(0.045F, values[3]);
assertEquals(false, values[4]);
assertEquals(null, values[5]);
assertArrayEquals("binary".getBytes(StandardCharsets.UTF_8), (byte[]) values[6]);
assertArrayEquals("fixed".getBytes(StandardCharsets.UTF_8), (byte[]) values[7]);
assertEquals(map, values[8]);
assertArrayEquals(new Object[] {1L, 2L}, (Object[]) values[9]);
final Map<String, Object> accountValues = new HashMap<>();
accountValues.put("accountName", "Checking");
accountValues.put("accountId", 83L);
final List<RecordField> accountRecordFields = new ArrayList<>();
accountRecordFields.add(new RecordField("accountId", RecordFieldType.LONG.getDataType()));
accountRecordFields.add(new RecordField("accountName", RecordFieldType.STRING.getDataType()));
final RecordSchema accountRecordSchema = new SimpleRecordSchema(accountRecordFields);
final Record mapRecord = new MapRecord(accountRecordSchema, accountValues);
assertEquals(mapRecord, values[10]);
assertNull(values[11]);
assertEquals(10_000_000.0D, values[12]);
final Map<String, Object> dogMap = new HashMap<>();
dogMap.put("dogName", "Fido");
dogMap.put("dogTailLength", 14);
final List<RecordField> dogRecordFields = new ArrayList<>();
dogRecordFields.add(new RecordField("dogTailLength", RecordFieldType.INT.getDataType()));
dogRecordFields.add(new RecordField("dogName", RecordFieldType.STRING.getDataType()));
final RecordSchema dogRecordSchema = new SimpleRecordSchema(dogRecordFields);
final Record dogRecord = new MapRecord(dogRecordSchema, dogMap);
assertEquals(dogRecord, values[13]);
final Map<String, Object> catMap = new HashMap<>();
catMap.put("catName", "Meow");
catMap.put("catTailLength", 1);
final List<RecordField> catRecordFields = new ArrayList<>();
catRecordFields.add(new RecordField("catTailLength", RecordFieldType.INT.getDataType()));
catRecordFields.add(new RecordField("catName", RecordFieldType.STRING.getDataType()));
final RecordSchema catRecordSchema = new SimpleRecordSchema(catRecordFields);
final Record catRecord = new MapRecord(catRecordSchema, catMap);
assertEquals(catRecord, values[14]);
}
}
public static enum Status {
GOOD, BAD;
}
}

Просмотреть файл

@ -0,0 +1,122 @@
/*
* 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.csv;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.RecordFieldType;
import org.apache.nifi.serialization.record.RecordSchema;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
public class TestCSVRecordReader {
private final DataType stringDataType = RecordFieldType.STRING.getDataType();
private final DataType doubleDataType = RecordFieldType.DOUBLE.getDataType();
private final DataType timeDataType = RecordFieldType.TIME.getDataType();
@Test
public void testSimpleParse() throws IOException, MalformedRecordException {
final Map<String, DataType> overrides = new HashMap<>();
overrides.put("balance", doubleDataType);
overrides.put("other", timeDataType);
try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/single-bank-account.csv"))) {
final CSVRecordReader reader = new CSVRecordReader(fis, null, overrides);
final RecordSchema schema = reader.getSchema();
verifyFields(schema);
final Object[] record = reader.nextRecord().getValues();
final Object[] expectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
Assert.assertArrayEquals(expectedValues, record);
assertNull(reader.nextRecord());
}
}
@Test
public void testMultipleRecords() throws IOException, MalformedRecordException {
final Map<String, DataType> overrides = new HashMap<>();
overrides.put("balance", doubleDataType);
try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/multi-bank-account.csv"))) {
final CSVRecordReader reader = new CSVRecordReader(fis, null, overrides);
final RecordSchema schema = reader.getSchema();
verifyFields(schema);
final Object[] firstRecord = reader.nextRecord().getValues();
final Object[] firstExpectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
Assert.assertArrayEquals(firstExpectedValues, firstRecord);
final Object[] secondRecord = reader.nextRecord().getValues();
final Object[] secondExpectedValues = new Object[] {"2", "Jane Doe", 4820.09D, "321 Your Street", "Your City", "NY", "33333", "USA"};
Assert.assertArrayEquals(secondExpectedValues, secondRecord);
assertNull(reader.nextRecord());
}
}
@Test
public void testExtraWhiteSpace() throws IOException, MalformedRecordException {
final Map<String, DataType> overrides = new HashMap<>();
overrides.put("balance", doubleDataType);
try (final InputStream fis = new FileInputStream(new File("src/test/resources/csv/extra-white-space.csv"))) {
final CSVRecordReader reader = new CSVRecordReader(fis, Mockito.mock(ComponentLog.class), overrides);
final RecordSchema schema = reader.getSchema();
verifyFields(schema);
final Object[] firstRecord = reader.nextRecord().getValues();
final Object[] firstExpectedValues = new Object[] {"1", "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"};
Assert.assertArrayEquals(firstExpectedValues, firstRecord);
final Object[] secondRecord = reader.nextRecord().getValues();
final Object[] secondExpectedValues = new Object[] {"2", "Jane Doe", 4820.09D, "321 Your Street", "Your City", "NY", "33333", "USA"};
Assert.assertArrayEquals(secondExpectedValues, secondRecord);
assertNull(reader.nextRecord());
}
}
private void verifyFields(final RecordSchema schema) {
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList("id", "name", "balance", "address", "city", "state", "zipCode", "country");
assertEquals(expectedFieldNames, fieldNames);
final List<DataType> dataTypes = schema.getDataTypes();
final List<DataType> expectedDataTypes = Arrays.asList(stringDataType, stringDataType, doubleDataType,
stringDataType, stringDataType, stringDataType, stringDataType, stringDataType);
assertEquals(expectedDataTypes, dataTypes);
}
}

Просмотреть файл

@ -0,0 +1,121 @@
/*
* 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.csv;
import static org.junit.Assert.assertEquals;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
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.RecordSet;
import org.junit.Test;
public class TestWriteCSVResult {
@Test
public void testDataTypes() throws IOException {
final WriteCSVResult result = new WriteCSVResult(RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
final StringBuilder headerBuilder = new StringBuilder();
final List<RecordField> fields = new ArrayList<>();
for (final RecordFieldType fieldType : RecordFieldType.values()) {
if (fieldType == RecordFieldType.CHOICE) {
final List<DataType> possibleTypes = new ArrayList<>();
possibleTypes.add(RecordFieldType.INT.getDataType());
possibleTypes.add(RecordFieldType.LONG.getDataType());
fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType(possibleTypes)));
} else {
fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
}
headerBuilder.append('"').append(fieldType.name().toLowerCase()).append('"').append(",");
}
final RecordSchema schema = new SimpleRecordSchema(fields);
final long now = System.currentTimeMillis();
final Map<String, Object> valueMap = new HashMap<>();
valueMap.put("string", "string");
valueMap.put("boolean", true);
valueMap.put("byte", (byte) 1);
valueMap.put("char", 'c');
valueMap.put("short", (short) 8);
valueMap.put("int", 9);
valueMap.put("bigint", BigInteger.valueOf(8L));
valueMap.put("long", 8L);
valueMap.put("float", 8.0F);
valueMap.put("double", 8.0D);
valueMap.put("date", new Date(now));
valueMap.put("time", new Time(now));
valueMap.put("timestamp", new Timestamp(now));
valueMap.put("object", null);
valueMap.put("choice", 48L);
valueMap.put("array", null);
final Record record = new MapRecord(schema, valueMap);
final RecordSet rs = RecordSet.of(schema, record);
final String output;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
result.write(rs, baos);
output = new String(baos.toByteArray(), StandardCharsets.UTF_8);
}
headerBuilder.deleteCharAt(headerBuilder.length() - 1);
final String headerLine = headerBuilder.toString();
final String[] splits = output.split("\n");
assertEquals(2, splits.length);
assertEquals(headerLine, splits[0]);
final String values = splits[1];
final StringBuilder expectedBuilder = new StringBuilder();
expectedBuilder.append("\"string\",\"true\",\"1\",\"c\",\"8\",\"9\",\"8\",\"8\",\"8.0\",\"8.0\",");
final String dateValue = new SimpleDateFormat(RecordFieldType.DATE.getDefaultFormat()).format(now);
final String timeValue = new SimpleDateFormat(RecordFieldType.TIME.getDefaultFormat()).format(now);
final String timestampValue = new SimpleDateFormat(RecordFieldType.TIMESTAMP.getDefaultFormat()).format(now);
expectedBuilder.append('"').append(dateValue).append('"').append(',');
expectedBuilder.append('"').append(timeValue).append('"').append(',');
expectedBuilder.append('"').append(timestampValue).append('"').append(',');
expectedBuilder.append(",\"48\",");
final String expectedValues = expectedBuilder.toString();
assertEquals(expectedValues, values);
}
}

Просмотреть файл

@ -0,0 +1,190 @@
/*
* 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.grok;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.record.Record;
import org.junit.Test;
import io.thekraken.grok.api.Grok;
import io.thekraken.grok.api.exception.GrokException;
public class TestGrokRecordReader {
@Test
public void testParseSingleLineLogMessages() throws GrokException, IOException, MalformedRecordException {
try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/single-line-log-messages.txt"))) {
final Grok grok = new Grok();
grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} %{GREEDYDATA:message}");
final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, Collections.emptyMap());
final String[] logLevels = new String[] {"INFO", "WARN", "ERROR", "FATAL", "FINE"};
final String[] messages = new String[] {"Test Message 1", "Red", "Green", "Blue", "Yellow"};
for (int i = 0; i < logLevels.length; i++) {
final Object[] values = deserializer.nextRecord().getValues();
assertNotNull(values);
assertEquals(4, values.length); // values[] contains 4 elements: timestamp, level, message, STACK_TRACE
assertEquals("2016-11-08 21:24:23,029", values[0]);
assertEquals(logLevels[i], values[1]);
assertEquals(messages[i], values[2]);
assertNull(values[3]);
}
assertNull(deserializer.nextRecord());
}
}
@Test
public void testParseEmptyMessageWithStackTrace() throws GrokException, IOException, MalformedRecordException {
final Grok grok = new Grok();
grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}");
final String msg = "2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.LoggerClass \n"
+ "org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces";
final InputStream bais = new ByteArrayInputStream(msg.getBytes(StandardCharsets.UTF_8));
final GrokRecordReader deserializer = new GrokRecordReader(bais, grok, Collections.emptyMap());
final Object[] values = deserializer.nextRecord().getValues();
assertNotNull(values);
assertEquals(6, values.length); // values[] contains 4 elements: timestamp, level, message, STACK_TRACE
assertEquals("2016-08-04 13:26:32,473", values[0]);
assertEquals("INFO", values[1]);
assertEquals("Leader Election Notification Thread-1", values[2]);
assertEquals("o.a.n.LoggerClass", values[3]);
assertEquals("", values[4]);
assertEquals("org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces", values[5]);
}
@Test
public void testParseNiFiSampleLog() throws IOException, GrokException, MalformedRecordException {
try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/nifi-log-sample.log"))) {
final Grok grok = new Grok();
grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}");
final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, Collections.emptyMap());
final String[] logLevels = new String[] {"INFO", "INFO", "INFO", "WARN", "WARN"};
for (int i = 0; i < logLevels.length; i++) {
final Object[] values = deserializer.nextRecord().getValues();
assertNotNull(values);
assertEquals(6, values.length); // values[] contains 6 elements: timestamp, level, thread, class, message, STACK_TRACE
assertEquals(logLevels[i], values[1]);
assertNull(values[5]);
}
assertNull(deserializer.nextRecord());
}
}
@Test
public void testParseNiFiSampleMultilineWithStackTrace() throws IOException, GrokException, MalformedRecordException {
try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/nifi-log-sample-multiline-with-stacktrace.log"))) {
final Grok grok = new Grok();
grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}?");
final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, Collections.emptyMap());
final String[] logLevels = new String[] {"INFO", "INFO", "ERROR", "WARN", "WARN"};
for (int i = 0; i < logLevels.length; i++) {
final Record record = deserializer.nextRecord();
final Object[] values = record.getValues();
assertNotNull(values);
assertEquals(6, values.length); // values[] contains 6 elements: timestamp, level, thread, class, message, STACK_TRACE
assertEquals(logLevels[i], values[1]);
if ("ERROR".equals(values[1])) {
final String msg = (String) values[4];
assertEquals("One\nTwo\nThree", msg);
assertNotNull(values[5]);
} else {
assertNull(values[5]);
}
}
assertNull(deserializer.nextRecord());
}
}
@Test
public void testParseStackTrace() throws GrokException, IOException, MalformedRecordException {
try (final InputStream fis = new FileInputStream(new File("src/test/resources/grok/error-with-stack-trace.log"))) {
final Grok grok = new Grok();
grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt");
grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} %{GREEDYDATA:message}");
final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, Collections.emptyMap());
final String[] logLevels = new String[] {"INFO", "ERROR", "INFO"};
final String[] messages = new String[] {"message without stack trace",
"Log message with stack trace",
"message without stack trace"};
for (int i = 0; i < logLevels.length; i++) {
final Object[] values = deserializer.nextRecord().getValues();
assertNotNull(values);
assertEquals(4, values.length); // values[] contains 4 elements: timestamp, level, message, STACK_TRACE
assertEquals(logLevels[i], values[1]);
assertEquals(messages[i], values[2]);
if (values[1].equals("ERROR")) {
final String stackTrace = (String) values[3];
assertNotNull(stackTrace);
assertTrue(stackTrace.startsWith("org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces"));
assertTrue(stackTrace.contains(" at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress("
+ "NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]"));
assertTrue(stackTrace.contains("Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces"));
assertTrue(stackTrace.contains("at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress("
+ "NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]"));
assertTrue(stackTrace.endsWith(" ... 12 common frames omitted"));
}
}
assertNull(deserializer.nextRecord());
}
}
}

Просмотреть файл

@ -0,0 +1,292 @@
/*
* 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.json;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
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.RecordSchema;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import com.jayway.jsonpath.JsonPath;
public class TestJsonPathRowRecordReader {
private final LinkedHashMap<String, JsonPath> allJsonPaths = new LinkedHashMap<>();
@Before
public void populateJsonPaths() {
allJsonPaths.clear();
allJsonPaths.put("id", JsonPath.compile("$.id"));
allJsonPaths.put("name", JsonPath.compile("$.name"));
allJsonPaths.put("balance", JsonPath.compile("$.balance"));
allJsonPaths.put("address", JsonPath.compile("$.address"));
allJsonPaths.put("city", JsonPath.compile("$.city"));
allJsonPaths.put("state", JsonPath.compile("$.state"));
allJsonPaths.put("zipCode", JsonPath.compile("$.zipCode"));
allJsonPaths.put("country", JsonPath.compile("$.country"));
}
@Test
public void testReadArray() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", "USA"}, secondRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testSingleJsonElement() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testElementWithNestedData() throws IOException, MalformedRecordException {
final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
jsonPaths.put("account", JsonPath.compile("$.account"));
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "account"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.RECORD});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
final Object[] simpleElements = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
Assert.assertArrayEquals(new Object[] {1, "John Doe", null, "123 My Street", "My City", "MS", "11111", "USA"}, simpleElements);
final Object lastElement = firstRecordValues[firstRecordValues.length - 1];
assertTrue(lastElement instanceof Record);
final Record record = (Record) lastElement;
assertEquals(42, record.getValue("id"));
assertEquals(4750.89D, record.getValue("balance"));
assertNull(reader.nextRecord());
}
}
@Test
public void testElementWithNestedArray() throws IOException, MalformedRecordException {
final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
jsonPaths.put("accounts", JsonPath.compile("$.accounts"));
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {
"id", "name", "balance", "address", "city", "state", "zipCode", "country", "accounts"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
final Object[] nonArrayValues = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
Assert.assertArrayEquals(new Object[] {1, "John Doe", null, "123 My Street", "My City", "MS", "11111", "USA"}, nonArrayValues);
final Object lastRecord = firstRecordValues[firstRecordValues.length - 1];
assertTrue(Object[].class.isAssignableFrom(lastRecord.getClass()));
final Object[] array = (Object[]) lastRecord;
assertEquals(2, array.length);
final Object firstElement = array[0];
assertTrue(firstElement instanceof Map);
final Map<?, ?> firstMap = (Map<?, ?>) firstElement;
assertEquals(42, firstMap.get("id"));
assertEquals(4750.89D, firstMap.get("balance"));
final Object secondElement = array[1];
assertTrue(secondElement instanceof Map);
final Map<?, ?> secondMap = (Map<?, ?>) secondElement;
assertEquals(43, secondMap.get("id"));
assertEquals(48212.38D, secondMap.get("balance"));
assertNull(reader.nextRecord());
}
}
@Test
public void testReadArrayDifferentSchemas() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(allJsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null}, secondRecordValues);
final Object[] thirdRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA"}, thirdRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testReadArrayDifferentSchemasWithOverride() throws IOException, MalformedRecordException {
final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
jsonPaths.put("address2", JsonPath.compile("$.address2"));
final Map<String, DataType> typeOverrides = Collections.singletonMap("address2", RecordFieldType.STRING.getDataType());
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, typeOverrides, in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "address2"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA", null}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null, null}, secondRecordValues);
final Object[] thirdRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA", "Apt. #12"}, thirdRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testPrimitiveTypeArrays() throws IOException, MalformedRecordException {
final LinkedHashMap<String, JsonPath> jsonPaths = new LinkedHashMap<>(allJsonPaths);
jsonPaths.put("accountIds", JsonPath.compile("$.accountIds"));
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/primitive-type-array.json"));
final JsonPathRowRecordReader reader = new JsonPathRowRecordReader(jsonPaths, Collections.emptyMap(), in, Mockito.mock(ComponentLog.class))) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "accountIds"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
final Object[] nonArrayValues = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89D, "123 My Street", "My City", "MS", "11111", "USA"}, nonArrayValues);
final Object lastRecord = firstRecordValues[firstRecordValues.length - 1];
assertNotNull(lastRecord);
assertTrue(Object[].class.isAssignableFrom(lastRecord.getClass()));
final Object[] array = (Object[]) lastRecord;
Assert.assertArrayEquals(new Object[] {1, 2, 3}, array);
assertNull(reader.nextRecord());
assertNull(reader.nextRecord());
}
}
}

Просмотреть файл

@ -0,0 +1,266 @@
/*
* 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.json;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.MalformedRecordException;
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.RecordSchema;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
public class TestJsonTreeRowRecordReader {
@Test
public void testReadArray() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", "USA"}, secondRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testSingleJsonElement() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testElementWithNestedData() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "address", "city", "state", "zipCode", "country", "account"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.RECORD});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
final Object[] allButLast = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
Assert.assertArrayEquals(new Object[] {1, "John Doe", "123 My Street", "My City", "MS", "11111", "USA"}, allButLast);
final Object last = firstRecordValues[firstRecordValues.length - 1];
assertTrue(Record.class.isAssignableFrom(last.getClass()));
final Record record = (Record) last;
assertEquals(42, record.getValue("id"));
assertEquals(4750.89, record.getValue("balance"));
assertNull(reader.nextRecord());
}
}
@Test
public void testElementWithNestedArray() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-element-nested-array.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {
"id", "name", "address", "city", "state", "zipCode", "country", "accounts"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.ARRAY});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
final Object[] nonArrayValues = Arrays.copyOfRange(firstRecordValues, 0, firstRecordValues.length - 1);
Assert.assertArrayEquals(new Object[] {1, "John Doe", "123 My Street", "My City", "MS", "11111", "USA"}, nonArrayValues);
final Object lastRecord = firstRecordValues[firstRecordValues.length - 1];
assertTrue(Object[].class.isAssignableFrom(lastRecord.getClass()));
assertNull(reader.nextRecord());
}
}
@Test
public void testReadArrayDifferentSchemas() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING,
RecordFieldType.DOUBLE, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null}, secondRecordValues);
final Object[] thirdRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA"}, thirdRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testReadArrayDifferentSchemasWithOverride() throws IOException, MalformedRecordException {
final Map<String, DataType> overrides = new HashMap<>();
overrides.put("address2", RecordFieldType.STRING.getDataType());
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-different-schemas.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), overrides)) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country", "address2"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA", null}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", 4820.09, "321 Your Street", "Your City", "NY", "33333", null, null}, secondRecordValues);
final Object[] thirdRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {3, "Jake Doe", 4751.89, "124 My Street", "My City", "MS", "11111", "USA", "Apt. #12"}, thirdRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testReadArrayDifferentSchemasWithOptionalElementOverridden() throws IOException, MalformedRecordException {
final Map<String, DataType> overrides = new HashMap<>();
overrides.put("balance", RecordFieldType.DOUBLE.getDataType());
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/bank-account-array-optional-balance.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), overrides)) {
final RecordSchema schema = reader.getSchema();
final List<String> fieldNames = schema.getFieldNames();
final List<String> expectedFieldNames = Arrays.asList(new String[] {"id", "name", "balance", "address", "city", "state", "zipCode", "country"});
assertEquals(expectedFieldNames, fieldNames);
final List<RecordFieldType> dataTypes = schema.getDataTypes().stream().map(dt -> dt.getFieldType()).collect(Collectors.toList());
final List<RecordFieldType> expectedTypes = Arrays.asList(new RecordFieldType[] {RecordFieldType.INT, RecordFieldType.STRING, RecordFieldType.DOUBLE, RecordFieldType.STRING,
RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING, RecordFieldType.STRING});
assertEquals(expectedTypes, dataTypes);
final Object[] firstRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {1, "John Doe", 4750.89, "123 My Street", "My City", "MS", "11111", "USA"}, firstRecordValues);
final Object[] secondRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {2, "Jane Doe", null, "321 Your Street", "Your City", "NY", "33333", "USA"}, secondRecordValues);
final Object[] thirdRecordValues = reader.nextRecord().getValues();
Assert.assertArrayEquals(new Object[] {3, "Jimmy Doe", null, "321 Your Street", "Your City", "NY", "33333", "USA"}, thirdRecordValues);
assertNull(reader.nextRecord());
}
}
@Test
public void testReadUnicodeCharacters() throws IOException, MalformedRecordException {
try (final InputStream in = new FileInputStream(new File("src/test/resources/json/json-with-unicode.json"));
final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), Collections.emptyMap())) {
final Object[] firstRecordValues = reader.nextRecord().getValues();
final Object secondValue = firstRecordValues[1];
assertTrue(secondValue instanceof Long);
assertEquals(832036744985577473L, secondValue);
final Object unicodeValue = firstRecordValues[2];
assertEquals("\u3061\u3083\u6ce3\u304d\u305d\u3046", unicodeValue);
assertNull(reader.nextRecord());
}
}
}

Просмотреть файл

@ -0,0 +1,102 @@
/*
* 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.json;
import static org.junit.Assert.assertEquals;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.math.BigInteger;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.DataType;
import org.apache.nifi.serialization.record.MapRecord;
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.RecordSet;
import org.junit.Test;
import org.mockito.Mockito;
public class TestWriteJsonResult {
@Test
public void testDataTypes() throws IOException, ParseException {
final WriteJsonResult writer = new WriteJsonResult(Mockito.mock(ComponentLog.class), true, RecordFieldType.DATE.getDefaultFormat(),
RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat());
final List<RecordField> fields = new ArrayList<>();
for (final RecordFieldType fieldType : RecordFieldType.values()) {
if (fieldType == RecordFieldType.CHOICE) {
final List<DataType> possibleTypes = new ArrayList<>();
possibleTypes.add(RecordFieldType.INT.getDataType());
possibleTypes.add(RecordFieldType.LONG.getDataType());
fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType(possibleTypes)));
} else {
fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType()));
}
}
final RecordSchema schema = new SimpleRecordSchema(fields);
final long time = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS").parse("2017/01/01 17:00:00.000").getTime();
final Map<String, Object> valueMap = new LinkedHashMap<>();
valueMap.put("string", "string");
valueMap.put("boolean", true);
valueMap.put("byte", (byte) 1);
valueMap.put("char", 'c');
valueMap.put("short", (short) 8);
valueMap.put("int", 9);
valueMap.put("bigint", BigInteger.valueOf(8L));
valueMap.put("long", 8L);
valueMap.put("float", 8.0F);
valueMap.put("double", 8.0D);
valueMap.put("date", new Date(time));
valueMap.put("time", new Time(time));
valueMap.put("timestamp", new Timestamp(time));
valueMap.put("record", null);
valueMap.put("array", null);
valueMap.put("choice", 48L);
final Record record = new MapRecord(schema, valueMap);
final RecordSet rs = RecordSet.of(schema, record);
final String output;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
writer.write(rs, baos);
output = baos.toString();
}
final String expected = new String(Files.readAllBytes(Paths.get("src/test/resources/json/output/dataTypes.json")));
assertEquals(expected, output);
}
}

Просмотреть файл

@ -0,0 +1,9 @@
id, name, balance, address, city, state, zipCode, country
1, John Doe, "4750.89", "123 My Street", My City, MS, 11111, USA
2, Jane Doe, 4820.09, 321 Your Street, Your City, NY, 33333, USA
1 id name balance address city state zipCode country
2 1 John Doe 4750.89 123 My Street My City MS 11111 USA
3 2 Jane Doe 4820.09 321 Your Street Your City NY 33333 USA

Просмотреть файл

@ -0,0 +1,3 @@
id, name, balance, address, city, state, zipCode, country
1, John Doe, "4750.89", "123 My Street", My City, MS, 11111, USA
2, Jane Doe, 4820.09, 321 Your Street, Your City, NY, 33333, USA
1 id name balance address city state zipCode country
2 1 John Doe 4750.89 123 My Street My City MS 11111 USA
3 2 Jane Doe 4820.09 321 Your Street Your City NY 33333 USA

Просмотреть файл

@ -0,0 +1,2 @@
id, name, balance, address, city, state, zipCode, country
1, John Doe, "4750.89", "123 My Street", My City, MS, 11111, USA
1 id name balance address city state zipCode country
2 1 John Doe 4750.89 123 My Street My City MS 11111 USA

Просмотреть файл

@ -0,0 +1,25 @@
2016-11-23 16:00:00,000 INFO message without stack trace
2016-11-23 16:00:02,689 ERROR Log message with stack trace
org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
... 12 common frames omitted
2016-11-23 16:05:00,000 INFO message without stack trace

Просмотреть файл

@ -0,0 +1,29 @@
2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'
2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 This node is no longer leader for role 'Cluster Coordinator'
2016-08-04 13:26:32,474 ERROR [Leader Election Notification Thread-2] o.apache.nifi.controller.FlowController One
Two
Three
org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_45]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_45]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
Caused by: org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
at org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.getElectedActiveCoordinatorAddress(NodeClusterCoordinator.java:185) [nifi-framework-cluster-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
... 12 common frames omitted
2016-08-04 13:26:35,475 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
2016-08-04 13:26:35,479 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3007 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.

Просмотреть файл

@ -0,0 +1,5 @@
2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 has been interrupted; no longer leader for role 'Cluster Coordinator'
2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.c.l.e.CuratorLeaderElectionManager org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager$ElectionListener@1fa27ea5 This node is no longer leader for role 'Cluster Coordinator'
2016-08-04 13:26:32,474 INFO [Leader Election Notification Thread-2] o.apache.nifi.controller.FlowController This node is no longer Primary Node
2016-08-04 13:26:35,475 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3008 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.
2016-08-04 13:26:35,479 WARN [Curator-Framework-0] org.apache.curator.ConnectionState Connection attempt unsuccessful after 3007 (greater than max timeout of 3000). Resetting connection and trying again with a new connection.

Просмотреть файл

@ -0,0 +1,5 @@
2016-11-08 21:24:23,029 INFO Test Message 1
2016-11-08 21:24:23,029 WARN Red
2016-11-08 21:24:23,029 ERROR Green
2016-11-08 21:24:23,029 FATAL Blue
2016-11-08 21:24:23,029 FINE Yellow

Просмотреть файл

@ -0,0 +1,30 @@
[
{
"id": 1,
"name": "John Doe",
"balance": 4750.89,
"address": "123 My Street",
"city": "My City",
"state": "MS",
"zipCode": "11111",
"country": "USA"
}, {
"id": 2,
"name": "Jane Doe",
"balance": 4820.09,
"address": "321 Your Street",
"city": "Your City",
"state": "NY",
"zipCode": "33333"
}, {
"id": 3,
"name": "Jake Doe",
"balance": 4751.89,
"address": "124 My Street",
"address2": "Apt. #12",
"city": "My City",
"state": "MS",
"zipCode": "11111",
"country": "USA"
}
]

Просмотреть файл

@ -0,0 +1,29 @@
[
{
"id": 1,
"name": "John Doe",
"balance": 4750.89,
"address": "123 My Street",
"city": "My City",
"state": "MS",
"zipCode": "11111",
"country": "USA"
}, {
"id": 2,
"name": "Jane Doe",
"balance": null,
"address": "321 Your Street",
"city": "Your City",
"state": "NY",
"zipCode": "33333",
"country": "USA"
}, {
"id": 3,
"name": "Jimmy Doe",
"address": "321 Your Street",
"city": "Your City",
"state": "NY",
"zipCode": "33333",
"country": "USA"
}
]

Просмотреть файл

@ -0,0 +1,21 @@
[
{
"id": 1,
"name": "John Doe",
"balance": 4750.89,
"address": "123 My Street",
"city": "My City",
"state": "MS",
"zipCode": "11111",
"country": "USA"
}, {
"id": 2,
"name": "Jane Doe",
"balance": 4820.09,
"address": "321 Your Street",
"city": "Your City",
"state": "NY",
"zipCode": "33333",
"country": "USA"
}
]

Некоторые файлы не были показаны из-за слишком большого количества измененных файлов Показать больше