From 8d20b82095857e06ee58ae892039a5f10613b125 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 7 Apr 2015 14:16:54 -0400 Subject: [PATCH 01/16] NIFI-492: When attempting to get a connection from the pool, if we create a new one and encounter an error, should ensure we close the new connection --- .../client/socket/EndpointConnectionPool.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java index daf52b4f07..1a6dfd510a 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java @@ -314,6 +314,11 @@ public class EndpointConnectionPool { if ( protocol.isDestinationFull() ) { logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer); penalize(peer, penalizationMillis); + try { + peer.close(); + } catch (final IOException ioe) { + } + continue; } else if ( protocol.isPortInvalid() ) { penalize(peer, penalizationMillis); @@ -359,6 +364,15 @@ public class EndpointConnectionPool { } } } while ( connection == null || codec == null || commsSession == null || protocol == null ); + } catch (final Throwable t) { + if ( commsSession != null ) { + try { + commsSession.close(); + } catch (final IOException ioe) { + } + } + + throw t; } finally { if ( !addBack.isEmpty() ) { connectionQueue.addAll(addBack); From 7819afbefd980ce68f43093302997024926d9f51 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Wed, 8 Apr 2015 13:38:33 -0400 Subject: [PATCH 02/16] NIFI-495: Fixed handling of FlowFiles if destination full by rolling back session --- .../java/org/apache/nifi/remote/StandardRemoteGroupPort.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java index 740e405b87..69ba0fd4f8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java @@ -171,6 +171,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort { this.targetRunning.set(false); final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port is not in a valid state", this, url); logger.error(message); + session.rollback(); remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); return; } catch (final UnknownPortException e) { @@ -178,6 +179,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort { this.targetExists.set(false); final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port no longer exists", this, url); logger.error(message); + session.rollback(); remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); return; } catch (final IOException e) { @@ -186,13 +188,14 @@ public class StandardRemoteGroupPort extends RemoteGroupPort { if ( logger.isDebugEnabled() ) { logger.error("", e); } + session.rollback(); remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - session.rollback(); return; } if ( transaction == null ) { logger.debug("{} Unable to create transaction to communicate with; all peers must be penalized, so yielding context", this); + session.rollback(); context.yield(); return; } From abd279c1e01e3aea3332e6b378701554722317e6 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Wed, 8 Apr 2015 13:41:34 -0400 Subject: [PATCH 03/16] NIFI-488: Redirect nifi output streams and redirect bootstrap log messages to file --- .../org/apache/nifi/bootstrap/RunNiFi.java | 131 +++++++++++++++--- .../src/main/resources/conf/bootstrap.conf | 5 + 2 files changed, 119 insertions(+), 17 deletions(-) diff --git a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java index 28a9b71028..d25df97025 100644 --- a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java +++ b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java @@ -45,9 +45,10 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.logging.ConsoleHandler; +import java.util.logging.FileHandler; import java.util.logging.Handler; import java.util.logging.Level; +import java.util.logging.SimpleFormatter; /** @@ -92,16 +93,82 @@ public class RunNiFi { private final java.util.logging.Logger logger; - public RunNiFi(final File bootstrapConfigFile, final boolean verbose) { + public RunNiFi(final File bootstrapConfigFile, final boolean verbose) throws IOException { this.bootstrapConfigFile = bootstrapConfigFile; logger = java.util.logging.Logger.getLogger("Bootstrap"); + + final Properties bootstrapProps = new Properties(); + try (final InputStream configIn = new FileInputStream(bootstrapConfigFile)) { + bootstrapProps.load(configIn); + } + + String logFilename = bootstrapProps.getProperty("bootstrap.log.file"); + if ( logFilename == null ) { + logFilename = "./logs/bootstrap.log"; + } + + File logFile = new File(logFilename); + if ( !logFile.isAbsolute() ) { + final File workDir = getDefaultWorkingDirectory(); + logFile = new File(workDir, logFilename); + } + + final File logFileDir = logFile.getParentFile(); + final Handler fileHandler; + if ( logFileDir.exists() || logFileDir.mkdirs() ) { + final int maxSize = getIntProp(bootstrapProps, "bootstrap.log.max.bytes", 1024 * 1024 * 10); // 10 MB + final int numFiles = getIntProp(bootstrapProps, "bootstrap.log.count", 10); + + fileHandler = new FileHandler(logFile.getAbsolutePath(), maxSize, numFiles, true); + fileHandler.setFormatter(new SimpleFormatter()); + logger.addHandler(fileHandler); + } else { + fileHandler = null; + logger.severe("Could not create log file directory " + logFileDir + ". Will not log bootstrap info to file or redirect NiFi standard out to file"); + } + if ( verbose ) { logger.info("Enabling Verbose Output"); logger.setLevel(Level.FINE); - final Handler handler = new ConsoleHandler(); - handler.setLevel(Level.FINE); - logger.addHandler(handler); + + for ( final Handler handler : logger.getHandlers() ) { + handler.setLevel(Level.FINE); + } + } + } + + + private File getLogFile() throws IOException { + final Properties bootstrapProps = new Properties(); + try (final InputStream configIn = new FileInputStream(bootstrapConfigFile)) { + bootstrapProps.load(configIn); + } + + String logFilename = bootstrapProps.getProperty("bootstrap.log.file"); + if ( logFilename == null ) { + logFilename = "./logs/bootstrap.log"; + } + + File logFile = new File(logFilename); + if ( !logFile.isAbsolute() ) { + final File workDir = getDefaultWorkingDirectory(); + logFile = new File(workDir, logFilename); + } + + return logFile; + } + + private static int getIntProp(final Properties properties, final String name, final int defaultValue) { + String propVal = properties.getProperty(name); + if ( propVal == null || propVal.trim().isEmpty() ) { + return defaultValue; + } + + try { + return Integer.parseInt(propVal.trim()); + } catch (final NumberFormatException nfe) { + throw new NumberFormatException("Expected bootstrap property '" + name + "' to be an integer but found value: " + propVal); } } @@ -581,6 +648,35 @@ public class RunNiFi { } } + private void redirectOutput(final Process process) { + redirectStreamToLogs(process.getInputStream()); + redirectStreamToLogs(process.getErrorStream()); + } + + private void redirectStreamToLogs(final InputStream in) { + final Thread t = new Thread(new Runnable() { + @Override + public void run() { + try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { + String line; + while ((line = reader.readLine()) != null) { + logger.info(line); + } + } catch (IOException e) { + logger.warning("Failed to read output of NiFi console: " + e); + } + } + }); + t.setDaemon(true); + t.start(); + } + + private File getDefaultWorkingDirectory() { + final File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile(); + final File binDir = bootstrapConfigAbsoluteFile.getParentFile(); + return binDir.getParentFile(); + } + @SuppressWarnings({ "rawtypes", "unchecked" }) public void start(final boolean monitor) throws IOException, InterruptedException { final Integer port = getCurrentPort(); @@ -590,7 +686,6 @@ public class RunNiFi { } final ProcessBuilder builder = new ProcessBuilder(); - if ( !bootstrapConfigFile.exists() ) { throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath()); } @@ -604,18 +699,17 @@ public class RunNiFi { props.putAll( (Map) properties ); final String specifiedWorkingDir = props.get("working.dir"); - if ( specifiedWorkingDir != null ) { - builder.directory(new File(specifiedWorkingDir)); - } - - final File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile(); - final File binDir = bootstrapConfigAbsoluteFile.getParentFile(); - final File workingDir = binDir.getParentFile(); - + final File workingDir = getDefaultWorkingDirectory(); if ( specifiedWorkingDir == null ) { builder.directory(workingDir); + } else { + builder.directory(new File(specifiedWorkingDir)); } + final File logDir = getLogFile().getParentFile(); + builder.redirectError(new File(logDir, "nifi.err")); + builder.redirectOutput(new File(logDir, "nifi.out")); + final String libFilename = replaceNull(props.get("lib.dir"), "./lib").trim(); File libDir = getFile(libFilename, workingDir); @@ -738,14 +832,15 @@ public class RunNiFi { try { gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown); } catch (final NumberFormatException nfe) { - throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); + throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); } if ( gracefulShutdownSeconds < 0 ) { - throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); + throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); } Process process = builder.start(); + redirectOutput(process); Long pid = getPid(process); if ( pid != null ) { nifiPid = pid; @@ -776,7 +871,8 @@ public class RunNiFi { if (autoRestartNiFi) { logger.warning("Apache NiFi appears to have died. Restarting..."); process = builder.start(); - + redirectOutput(process); + pid = getPid(process); if ( pid != null ) { nifiPid = pid; @@ -802,6 +898,7 @@ public class RunNiFi { } } else { final Process process = builder.start(); + redirectOutput(process); final Long pid = getPid(process); if ( pid != null ) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf index c1536d829f..e87bde2673 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf @@ -21,6 +21,11 @@ java=java # Username to use when running NiFi. This value will be ignored on Windows. run.as= +# Bootstrap logger info +bootstrap.log.file=logs/nifi-bootstrap.log +bootstrap.log.max.bytes=10485760 +bootstrap.log.count=10 + # Configure where NiFi's lib and conf directories live lib.dir=./lib conf.dir=./conf From e8fde859141e55fd13a1eaae77f4705e61f6071d Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 9 Apr 2015 09:31:20 -0400 Subject: [PATCH 04/16] NIFI-496: Only update the updatedAttributes map if a value is changed --- .../repository/StandardRepositoryRecord.java | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java index 1fde9aabd8..6ecb9910e7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java @@ -116,12 +116,23 @@ public class StandardRepositoryRecord implements RepositoryRecord { public void setWorking(final FlowFileRecord flowFile, final String attributeKey, final String attributeValue) { workingFlowFileRecord = flowFile; - updatedAttributes.put(attributeKey, attributeValue); + + // If setting attribute to same value as original, don't add to updated attributes + final String currentValue = originalAttributes.get(attributeKey); + if ( currentValue == null || !currentValue.equals(attributeValue) ) { + updatedAttributes.put(attributeKey, attributeValue); + } } public void setWorking(final FlowFileRecord flowFile, final Map updatedAttribs) { workingFlowFileRecord = flowFile; - updatedAttributes.putAll(updatedAttribs); + + for ( final Map.Entry entry : updatedAttribs.entrySet() ) { + final String currentValue = originalAttributes.get(entry.getKey()); + if ( currentValue == null || !currentValue.equals(entry.getValue()) ) { + updatedAttributes.put(entry.getKey(), entry.getValue()); + } + } } @Override From 45416dc66bc212bacf5a8429e4ae2ae880c0672c Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 9 Apr 2015 17:54:33 -0400 Subject: [PATCH 05/16] NIFI-506: Initial import of HL7 work --- .../nifi-hl7-query-language/.gitignore | 3 + .../nifi-hl7-query-language/pom.xml | 115 +++++ .../nifi/hl7/query/antlr/HL7QueryLexer.g | 156 +++++++ .../nifi/hl7/query/antlr/HL7QueryParser.g | 91 ++++ .../org/apache/nifi/hl7/hapi/EmptyField.java | 37 ++ .../org/apache/nifi/hl7/hapi/HapiField.java | 83 ++++ .../org/apache/nifi/hl7/hapi/HapiMessage.java | 94 ++++ .../org/apache/nifi/hl7/hapi/HapiSegment.java | 69 +++ .../nifi/hl7/hapi/SingleValueField.java | 42 ++ .../org/apache/nifi/hl7/io/HL7Reader.java | 27 ++ .../hl7/io/exception/InvalidHL7Exception.java | 40 ++ .../apache/nifi/hl7/model/HL7Component.java | 24 + .../org/apache/nifi/hl7/model/HL7Field.java | 21 + .../org/apache/nifi/hl7/model/HL7Message.java | 27 ++ .../org/apache/nifi/hl7/model/HL7Segment.java | 27 ++ .../apache/nifi/hl7/query/Declaration.java | 29 ++ .../org/apache/nifi/hl7/query/HL7Query.java | 412 ++++++++++++++++++ .../apache/nifi/hl7/query/QueryResult.java | 29 ++ .../org/apache/nifi/hl7/query/ResultHit.java | 25 ++ .../org/apache/nifi/hl7/query/Selection.java | 37 ++ .../hl7/query/evaluator/BooleanEvaluator.java | 24 + .../nifi/hl7/query/evaluator/Evaluator.java | 27 ++ .../hl7/query/evaluator/IntegerEvaluator.java | 26 ++ .../hl7/query/evaluator/StringEvaluator.java | 25 ++ .../AbstractComparisonEvaluator.java | 106 +++++ .../comparison/AbstractNumericComparison.java | 67 +++ .../evaluator/comparison/EqualsEvaluator.java | 32 ++ .../comparison/GreaterThanEvaluator.java | 34 ++ .../GreaterThanOrEqualEvaluator.java | 34 ++ .../evaluator/comparison/IsNullEvaluator.java | 69 +++ .../comparison/LessThanEvaluator.java | 31 ++ .../comparison/LessThanOrEqualEvaluator.java | 31 ++ .../comparison/NotEqualsEvaluator.java | 32 ++ .../evaluator/comparison/NotEvaluator.java | 36 ++ .../comparison/NotNullEvaluator.java | 65 +++ .../literal/IntegerLiteralEvaluator.java | 36 ++ .../literal/StringLiteralEvaluator.java | 35 ++ .../query/evaluator/logic/AndEvaluator.java | 43 ++ .../query/evaluator/logic/OrEvaluator.java | 43 ++ .../message/DeclaredReferenceEvaluator.java | 42 ++ .../query/evaluator/message/DotEvaluator.java | 88 ++++ .../evaluator/message/FieldEvaluator.java | 67 +++ .../evaluator/message/MessageEvaluator.java | 34 ++ .../evaluator/message/SegmentEvaluator.java | 51 +++ .../exception/HL7QueryParsingException.java | 37 ++ .../nifi/hl7/query/result/MissedResult.java | 56 +++ .../hl7/query/result/StandardQueryResult.java | 69 +++ .../hl7/query/result/StandardResultHit.java | 41 ++ .../apache/nifi/hl7/query/TestHL7Query.java | 352 +++++++++++++++ .../src/test/resources/hyperglycemia | 5 + .../src/test/resources/hypoglycemia | 5 + .../src/test/resources/metabolic-panel | 23 + .../resources/unsolicited-vaccine-update-long | 16 + .../unsolicited-vaccine-update-short | 4 + .../src/test/resources/vaccine-query | 3 + .../src/test/resources/vaers-message-long | 60 +++ .../nifi-hl7-bundle/nifi-hl7-nar/pom.xml | 36 ++ .../nifi-hl7-processors/.gitignore | 1 + .../nifi-hl7-processors/pom.xml | 106 +++++ .../processors/hl7/ExtractHL7Attributes.java | 247 +++++++++++ .../apache/nifi/processors/hl7/RouteHL7.java | 217 +++++++++ .../org.apache.nifi.processor.Processor | 16 + .../hl7/TestExtractHL7Attributes.java | 48 ++ .../src/test/resources/1.hl7 | 16 + .../src/test/resources/hypoglycemia.hl7 | 5 + nifi/nifi-nar-bundles/nifi-hl7-bundle/pom.xml | 33 ++ 66 files changed, 3862 insertions(+) create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/.gitignore create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/pom.xml create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query create mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/.gitignore create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 create mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/pom.xml diff --git a/nifi/nifi-commons/nifi-hl7-query-language/.gitignore b/nifi/nifi-commons/nifi-hl7-query-language/.gitignore new file mode 100644 index 0000000000..e91d5c41bb --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/.gitignore @@ -0,0 +1,3 @@ +/target/ +/target/ +/target/ diff --git a/nifi/nifi-commons/nifi-hl7-query-language/pom.xml b/nifi/nifi-commons/nifi-hl7-query-language/pom.xml new file mode 100644 index 0000000000..447a88b84d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/pom.xml @@ -0,0 +1,115 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-commons + 0.1.0-incubating-SNAPSHOT + + + nifi-hl7-query-language + jar + + NiFi Health Level 7 (HL7) Query Language + + + + + maven-compiler-plugin + + 1.7 + 1.7 + + + + org.antlr + antlr3-maven-plugin + + + + antlr + + + + + + + + + + org.antlr + antlr-runtime + 3.5.2 + + + + + ca.uhn.hapi + hapi-base + 2.2 + + + ca.uhn.hapi + hapi-structures-v21 + 2.2 + + + ca.uhn.hapi + hapi-structures-v22 + 2.2 + + + ca.uhn.hapi + hapi-structures-v23 + 2.2 + + + ca.uhn.hapi + hapi-structures-v231 + 2.2 + + + ca.uhn.hapi + hapi-structures-v24 + 2.2 + + + ca.uhn.hapi + hapi-structures-v25 + 2.2 + + + ca.uhn.hapi + hapi-structures-v251 + 2.2 + + + ca.uhn.hapi + hapi-structures-v26 + 2.2 + + + + + junit + junit + test + + + diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g new file mode 100644 index 0000000000..7fe3386b4d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g @@ -0,0 +1,156 @@ +lexer grammar HL7QueryLexer; + +@header { + package org.apache.nifi.hl7.query.antlr; + import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +} + +@rulecatch { + catch(final Exception e) { + throw new HL7QueryParsingException(e); + } +} + +@members { + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } + + public void recover(RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } +} + + +// PUNCTUATION & SPECIAL CHARACTERS +WHITESPACE : (' '|'\t'|'\n'|'\r')+ { $channel = HIDDEN; }; +COMMENT : '#' ( ~('\n') )* '\n' { $channel = HIDDEN; }; + +LPAREN : '('; +RPAREN : ')'; +LBRACE : '{'; +RBRACE : '}'; +COLON : ':'; +COMMA : ','; +DOT : '.'; +SEMICOLON : ';'; + + + +// OPERATORS +EQUALS : '='; +NOT_EQUALS : '!='; +GT : '>'; +GE : '>='; +LT : '<'; +LE : '<='; +REGEX : 'MATCHES REGEX'; +LIKE : 'LIKE'; +IS_NULL : 'IS NULL'; +NOT_NULL : 'NOT NULL'; + + +// KEYWORDS +AND : 'AND'; +OR : 'OR'; +NOT : 'NOT'; + +TRUE : 'true'; +FALSE : 'false'; + +SELECT : 'select' | 'SELECT'; +DECLARE : 'declare' | 'DECLARE'; +OPTIONAL : 'optional' | 'OPTIONAL'; +REQUIRED : 'required' | 'REQUIRED'; +AS : 'as' | 'AS'; +WHERE : 'where' | 'WHERE'; + +MESSAGE : 'MESSAGE' | 'message'; +SEGMENT : 'SEGMENT' | 'segment'; + + +SEGMENT_NAME : LETTER ALPHA_NUMERIC ALPHA_NUMERIC; + + +NUMBER : ('0'..'9')+; +fragment LETTER : 'A'..'Z'; +fragment ALPHA_NUMERIC : 'A'..'Z' | '0'..'9'; + + +// STRINGS +STRING_LITERAL +@init{StringBuilder lBuf = new StringBuilder();} + : + ( + '"' + ( + escaped=ESC {lBuf.append(getText());} | + normal = ~( '"' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} + )* + '"' + ) + { + setText(lBuf.toString()); + } + | + ( + '\'' + ( + escaped=ESC {lBuf.append(getText());} | + normal = ~( '\'' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} + )* + '\'' + ) + { + setText(lBuf.toString()); + } + ; + + +fragment +ESC + : '\\' + ( + '"' { setText("\""); } + | '\'' { setText("\'"); } + | 'r' { setText("\r"); } + | 'n' { setText("\n"); } + | 't' { setText("\t"); } + | '\\' { setText("\\\\"); } + | nextChar = ~('"' | '\'' | 'r' | 'n' | 't' | '\\') + { + StringBuilder lBuf = new StringBuilder(); lBuf.append("\\\\").appendCodePoint(nextChar); setText(lBuf.toString()); + } + ) + ; + +IDENTIFIER : ( + ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n' | '0'..'9' | '.') + ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n' | '.')* + ); diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g new file mode 100644 index 0000000000..4d8d13c59a --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g @@ -0,0 +1,91 @@ +parser grammar HL7QueryParser; + +options { + output=AST; + tokenVocab=HL7QueryLexer; +} + +tokens { + QUERY; + DECLARATION; +} + +@header { + package org.apache.nifi.hl7.query.antlr; + import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +} + +@members { + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } + + public void recover(final RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } +} + + +declareClause : DECLARE^ declaration (COMMA! declaration)*; + +requiredOrOptional : REQUIRED | OPTIONAL; +declaration : IDENTIFIER AS requiredOrOptional SEGMENT_NAME -> + ^(DECLARATION IDENTIFIER requiredOrOptional SEGMENT_NAME); + + +selectClause : SELECT^ selectableClause; +selectableClause : selectable (COMMA! selectable)*; +selectable : (MESSAGE | ref | field)^ (AS! IDENTIFIER^)?; + + +whereClause : WHERE^ conditions; + +conditions : condition ((AND^ | OR^) condition)*; + +condition : NOT^ condition | LPAREN! conditions RPAREN! | evaluation; + +evaluation : expression + ( + unaryOperator^ + | (binaryOperator^ expression) + ); + +expression : (LPAREN! expr RPAREN!) | expr; +expr : ref | field | STRING_LITERAL | NUMBER; + +unaryOperator : IS_NULL | NOT_NULL; +binaryOperator : EQUALS | NOT_EQUALS | LT | GT | LE | GE; + +ref : (SEGMENT_NAME | IDENTIFIER); +field : ref DOT^ NUMBER + (DOT^ NUMBER (DOT^ NUMBER (DOT^ NUMBER)?)?)?; + + +query : declareClause? selectClause whereClause? EOF -> + ^(QUERY declareClause? selectClause whereClause?); diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java new file mode 100644 index 0000000000..be645e59c8 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java @@ -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.hl7.hapi; + +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Field; + +public class EmptyField implements HL7Field { + + @Override + public String getValue() { + return null; + } + + @Override + public List getComponents() { + return Collections.emptyList(); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java new file mode 100644 index 0000000000..056b6b677d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.hapi; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Field; + +import ca.uhn.hl7v2.model.Composite; +import ca.uhn.hl7v2.model.ExtraComponents; +import ca.uhn.hl7v2.model.Primitive; +import ca.uhn.hl7v2.model.Type; +import ca.uhn.hl7v2.model.Varies; +import ca.uhn.hl7v2.parser.EncodingCharacters; +import ca.uhn.hl7v2.parser.PipeParser; + +public class HapiField implements HL7Field, HL7Component { + private final String value; + private final List components; + + public HapiField(final Type type) { + this.value = PipeParser.encode(type, EncodingCharacters.defaultInstance()); + + final List componentList = new ArrayList<>(); + if ( type instanceof Composite ) { + final Composite composite = (Composite) type; + + for ( final Type component : composite.getComponents() ) { + componentList.add(new HapiField(component)); + } + } + + final ExtraComponents extra = type.getExtraComponents(); + if ( extra != null && extra.numComponents() > 0 ) { + final String singleFieldValue; + if ( type instanceof Primitive ) { + singleFieldValue = ((Primitive) type).getValue(); + } else { + singleFieldValue = this.value; + } + componentList.add(new SingleValueField(singleFieldValue)); + + for (int i=0; i < extra.numComponents(); i++) { + final Varies varies = extra.getComponent(i); + componentList.add(new HapiField(varies)); + } + } + + this.components = Collections.unmodifiableList(componentList); + } + + @Override + public String getValue() { + return value; + } + + @Override + public List getComponents() { + return components; + } + + @Override + public String toString() { + return value; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java new file mode 100644 index 0000000000..ddd28b2bf9 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java @@ -0,0 +1,94 @@ +/* + * 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.hl7.hapi; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.model.HL7Segment; + +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.model.Group; +import ca.uhn.hl7v2.model.Message; +import ca.uhn.hl7v2.model.Segment; +import ca.uhn.hl7v2.model.Structure; + +public class HapiMessage implements HL7Message { + private final Message message; + private final List allSegments; + private final Map> segmentMap; + + public HapiMessage(final Message message) throws HL7Exception { + this.message = message; + + allSegments = new ArrayList<>(); + populateSegments(message, allSegments); + + segmentMap = new HashMap<>(); + for ( final HL7Segment segment : allSegments ) { + final String segmentName = segment.getName(); + List segmentList = segmentMap.get(segmentName); + if ( segmentList == null ) { + segmentList = new ArrayList<>(); + segmentMap.put(segmentName, segmentList); + } + + segmentList.add(segment); + } + } + + private void populateSegments(final Group group, final List segments) throws HL7Exception { + for ( final String structureName : group.getNames() ) { + final Structure[] structures = group.getAll(structureName); + if ( group.isGroup(structureName) ) { + for ( final Structure structure : structures ) { + populateSegments((Group) structure, segments); + } + } else { + for ( final Structure structure : structures ) { + final Segment segment = (Segment) structure; + final HapiSegment hapiSegment = new HapiSegment(segment); + segments.add(hapiSegment); + } + } + } + } + + @Override + public List getSegments() { + return Collections.unmodifiableList(allSegments); + } + + @Override + public List getSegments(final String segmentType) { + final List segments = segmentMap.get(segmentType); + if ( segments == null ) { + return Collections.emptyList(); + } + + return Collections.unmodifiableList(segments); + } + + @Override + public String toString() { + return message.toString(); + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java new file mode 100644 index 0000000000..d50afdb960 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java @@ -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.hl7.hapi; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.model.HL7Segment; + +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.model.Segment; +import ca.uhn.hl7v2.model.Type; + +public class HapiSegment implements HL7Segment { + private final Segment segment; + private final List fields; + + public HapiSegment(final Segment segment) throws HL7Exception { + this.segment = segment; + + final List fieldList = new ArrayList<>(); + for (int i=1; i <= segment.numFields(); i++) { + final Type[] types = segment.getField(i); + + if ( types == null || types.length == 0 ) { + fieldList.add(new EmptyField()); + continue; + } + + for ( final Type type : types ) { + fieldList.add(new HapiField(type)); + } + } + + this.fields = Collections.unmodifiableList(fieldList); + } + + + @Override + public String getName() { + return segment.getName(); + } + + @Override + public List getFields() { + return fields; + } + + @Override + public String toString() { + return segment.toString(); + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java new file mode 100644 index 0000000000..ed99077b45 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java @@ -0,0 +1,42 @@ +/* + * 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.hl7.hapi; + +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Field; + +public class SingleValueField implements HL7Field { + private final String value; + + public SingleValueField(final String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + @Override + public List getComponents() { + return Collections.emptyList(); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java new file mode 100644 index 0000000000..e7b31a4c4a --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.io; + +import java.io.IOException; + +import org.apache.nifi.hl7.model.HL7Message; + +public interface HL7Reader { + + HL7Message nextMessage() throws IOException; + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java new file mode 100644 index 0000000000..669f40c6bc --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java @@ -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.hl7.io.exception; + +import java.io.IOException; + +public class InvalidHL7Exception extends IOException { + private static final long serialVersionUID = -5675416667224562441L; + + public InvalidHL7Exception() { + super(); + } + + public InvalidHL7Exception(String message, Throwable cause) { + super(message, cause); + } + + public InvalidHL7Exception(String message) { + super(message); + } + + public InvalidHL7Exception(Throwable cause) { + super(cause); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java new file mode 100644 index 0000000000..cf355041b1 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java @@ -0,0 +1,24 @@ +/* + * 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.hl7.model; + +import java.util.List; + +public interface HL7Component { + String getValue(); + List getComponents(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java new file mode 100644 index 0000000000..4086e581d1 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java @@ -0,0 +1,21 @@ +/* + * 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.hl7.model; + + +public interface HL7Field extends HL7Component { +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java new file mode 100644 index 0000000000..dd9c2a9ff4 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.model; + +import java.util.List; + +public interface HL7Message { + + List getSegments(); + + List getSegments(String segmentType); + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java new file mode 100644 index 0000000000..de5aaa1d4b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.model; + +import java.util.List; + +public interface HL7Segment { + + String getName(); + + List getFields(); + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java new file mode 100644 index 0000000000..0903cc8d30 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java @@ -0,0 +1,29 @@ +/* + * 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.hl7.query; + +import org.apache.nifi.hl7.model.HL7Message; + +public interface Declaration { + + String getAlias(); + + boolean isRequired(); + + Object getDeclaredValue(HL7Message message); + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java new file mode 100644 index 0000000000..a036106481 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java @@ -0,0 +1,412 @@ +/* + * 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.hl7.query; + +import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.*; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.antlr.runtime.ANTLRStringStream; +import org.antlr.runtime.CharStream; +import org.antlr.runtime.CommonTokenStream; +import org.antlr.runtime.tree.Tree; +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.EqualsEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.GreaterThanEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.GreaterThanOrEqualEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.IsNullEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.LessThanEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.LessThanOrEqualEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.NotEqualsEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.NotEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.NotNullEvaluator; +import org.apache.nifi.hl7.query.evaluator.literal.IntegerLiteralEvaluator; +import org.apache.nifi.hl7.query.evaluator.literal.StringLiteralEvaluator; +import org.apache.nifi.hl7.query.evaluator.logic.AndEvaluator; +import org.apache.nifi.hl7.query.evaluator.logic.OrEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.DeclaredReferenceEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.DotEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.MessageEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.SegmentEvaluator; +import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +import org.apache.nifi.hl7.query.result.MissedResult; +import org.apache.nifi.hl7.query.result.StandardQueryResult; + +import org.apache.nifi.hl7.query.antlr.HL7QueryLexer; +import org.apache.nifi.hl7.query.antlr.HL7QueryParser; + + +public class HL7Query { + private final Tree tree; + private final String query; + private final Set declarations = new HashSet<>(); + + private final List selections; + private final BooleanEvaluator whereEvaluator; + + private HL7Query(final Tree tree, final String query) { + this.tree = tree; + this.query = query; + + List select = null; + BooleanEvaluator where = null; + for (int i=0; i < tree.getChildCount(); i++) { + final Tree child = tree.getChild(i); + + switch (child.getType()) { + case DECLARE: + processDeclare(child); + break; + case SELECT: + select = processSelect(child); + break; + case WHERE: + where = processWhere(child); + break; + default: + throw new HL7QueryParsingException("Found unexpected clause at root level: " + tree.getText()); + } + } + + this.whereEvaluator = where; + this.selections = select; + } + + private void processDeclare(final Tree declare) { + for (int i=0; i < declare.getChildCount(); i++) { + final Tree declarationTree = declare.getChild(i); + + final String identifier = declarationTree.getChild(0).getText(); + final Tree requiredOrOptionalTree = declarationTree.getChild(1); + final boolean required = requiredOrOptionalTree.getType() == REQUIRED; + + final String segmentName = declarationTree.getChild(2).getText(); + + final Declaration declaration = new Declaration() { + @Override + public String getAlias() { + return identifier; + } + + @Override + public boolean isRequired() { + return required; + } + + @Override + public Object getDeclaredValue(final HL7Message message) { + if ( message == null ) { + return null; + } + + return message.getSegments(segmentName); + } + }; + + declarations.add(declaration); + } + } + + private List processSelect(final Tree select) { + final List selections = new ArrayList<>(); + + for (int i=0; i < select.getChildCount(); i++) { + final Tree selectable = select.getChild(i); + + final String alias = getSelectedName(selectable); + final Evaluator selectionEvaluator = buildReferenceEvaluator(selectable); + final Selection selection = new Selection(selectionEvaluator, alias); + selections.add(selection); + } + + return selections; + } + + + private String getSelectedName(final Tree selectable) { + if ( selectable.getChildCount() == 0 ) { + return selectable.getText(); + } else if (selectable.getType() == DOT ) { + return getSelectedName(selectable.getChild(0)) + "." + getSelectedName(selectable.getChild(1)); + } else { + return selectable.getChild(selectable.getChildCount() - 1).getText(); + } + } + + + private BooleanEvaluator processWhere(final Tree where) { + return buildBooleanEvaluator(where.getChild(0)); + } + + + private Evaluator buildReferenceEvaluator(final Tree tree) { + switch (tree.getType()) { + case MESSAGE: + return new MessageEvaluator(); + case SEGMENT_NAME: + return new SegmentEvaluator(new StringLiteralEvaluator(tree.getText())); + case IDENTIFIER: + return new DeclaredReferenceEvaluator(new StringLiteralEvaluator(tree.getText())); + case DOT: + final Tree firstChild = tree.getChild(0); + final Tree secondChild = tree.getChild(1); + return new DotEvaluator(buildReferenceEvaluator(firstChild), buildIntegerEvaluator(secondChild)); + case STRING_LITERAL: + return new StringLiteralEvaluator(tree.getText()); + case NUMBER: + return new IntegerLiteralEvaluator(Integer.parseInt(tree.getText())); + default: + throw new HL7QueryParsingException("Failed to build evaluator for " + tree.getText()); + } + } + + + private IntegerEvaluator buildIntegerEvaluator(final Tree tree) { + switch (tree.getType()) { + case NUMBER: + return new IntegerLiteralEvaluator(Integer.parseInt(tree.getText())); + default: + throw new HL7QueryParsingException("Failed to build Integer Evaluator for " + tree.getText()); + } + } + + + private BooleanEvaluator buildBooleanEvaluator(final Tree tree) { + // TODO: add Date comparisons + // LT/GT/GE/GE should allow for dates based on Field's Type + // BETWEEN + // DATE('2015/01/01') + // DATE('2015/01/01 12:00:00') + // DATE('24 HOURS AGO') + // DATE('YESTERDAY') + + switch (tree.getType()) { + case EQUALS: + return new EqualsEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case NOT_EQUALS: + return new NotEqualsEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case GT: + return new GreaterThanEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case LT: + return new LessThanEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case GE: + return new GreaterThanOrEqualEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case LE: + return new LessThanOrEqualEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case NOT: + return new NotEvaluator(buildBooleanEvaluator(tree.getChild(0))); + case AND: + return new AndEvaluator(buildBooleanEvaluator(tree.getChild(0)), buildBooleanEvaluator(tree.getChild(1))); + case OR: + return new OrEvaluator(buildBooleanEvaluator(tree.getChild(0)), buildBooleanEvaluator(tree.getChild(1))); + case IS_NULL: + return new IsNullEvaluator(buildReferenceEvaluator(tree.getChild(0))); + case NOT_NULL: + return new NotNullEvaluator(buildReferenceEvaluator(tree.getChild(0))); + default: + throw new HL7QueryParsingException("Cannot build boolean evaluator for '" + tree.getText() + "'"); + } + } + + + Tree getTree() { + return tree; + } + + public String getQuery() { + return query; + } + + @Override + public String toString() { + return "HL7Query[" + query + "]"; + } + + public static HL7Query compile(final String query) { + try { + final CommonTokenStream lexerTokenStream = createTokenStream(query); + final HL7QueryParser parser = new HL7QueryParser(lexerTokenStream); + final Tree tree = (Tree) parser.query().getTree(); + + return new HL7Query(tree, query); + } catch (final HL7QueryParsingException e) { + throw e; + } catch (final Exception e) { + throw new HL7QueryParsingException(e); + } + } + + private static CommonTokenStream createTokenStream(final String expression) throws HL7QueryParsingException { + final CharStream input = new ANTLRStringStream(expression); + final HL7QueryLexer lexer = new HL7QueryLexer(input); + return new CommonTokenStream(lexer); + } + + public List> getReturnTypes() { + final List> returnTypes = new ArrayList<>(); + + for ( final Selection selection : selections ) { + returnTypes.add( selection.getEvaluator().getType() ); + } + + return returnTypes; + } + + @SuppressWarnings("unchecked") + public QueryResult evaluate(final HL7Message message) { + + int totalIterations = 1; + final LinkedHashMap> possibleValueMap = new LinkedHashMap<>(); + for ( final Declaration declaration : declarations ) { + final Object value = declaration.getDeclaredValue(message); + if ( value == null && declaration.isRequired() ) { + return new MissedResult(selections); + } + + final List possibleValues; + if ( value instanceof List ) { + possibleValues = (List) value; + } else if ( value instanceof Collection ) { + possibleValues = new ArrayList((Collection) value); + } else { + possibleValues = new ArrayList<>(1); + possibleValues.add(value); + } + + if ( possibleValues.isEmpty() ) { + return new MissedResult(selections); + } + + possibleValueMap.put(declaration.getAlias(), possibleValues); + totalIterations *= possibleValues.size(); + } + + final Set> resultSet = new HashSet<>(); + for (int i=0; i < totalIterations; i++) { + final Map aliasValues = assignAliases(possibleValueMap, i); + + aliasValues.put(Evaluator.MESSAGE_KEY, message); + if (whereEvaluator == null || Boolean.TRUE.equals(whereEvaluator.evaluate(aliasValues))) { + final Map resultMap = new HashMap<>(); + + for ( final Selection selection : selections ) { + final Object value = selection.getEvaluator().evaluate(aliasValues); + resultMap.put(selection.getName(), value); + } + + resultSet.add(resultMap); + } + } + +// for ( final Declaration declaration : declarations ) { +// final Object value = declaration.getDeclaredValue(message); +// if ( value == null && declaration.isRequired() ) { +// return new MissedResult(selections); +// } +// objectMap.put(declaration.getAlias(), value); +// } +// +// if (whereEvaluator == null || Boolean.TRUE.equals(whereEvaluator.evaluate(objectMap))) { +// for ( final Selection selection : selections ) { +// final Object value = selection.getEvaluator().evaluate(objectMap); +// resultMap.put(selection.getName(), value); +// } +// } else { +// return new MissedResult(selections); +// } + + return new StandardQueryResult(selections, resultSet); + } + + + // assigns one of the possible values to each alias, based on which iteration this is. + // require LinkedHashMap just to be very clear and explicit that the order of the Map MUST be guaranteed + // between multiple invocations of this method. + // package protected for testing visibility +// static Map assignAliases(final LinkedHashMap> possibleValues, final int iteration) { +// final Map aliasMap = new HashMap<>(); +// +// int aliasIndex = possibleValues.size() - 1; +// for ( final Map.Entry> entry : possibleValues.entrySet() ) { +// final String alias = entry.getKey(); +// final List validValues = entry.getValue(); +// +// final int validValueIndex; +// if (aliasIndex > 0) { +// validValueIndex = iteration / aliasIndex; +// } else { +// validValueIndex = iteration; +// } +// +// final Object obj = validValues.get(validValueIndex % validValues.size()); +// aliasMap.put(alias, obj); +// +// aliasIndex--; +// } +// +// return aliasMap; +// } +// + + static Map assignAliases(final LinkedHashMap> possibleValues, final int iteration) { + final Map aliasMap = new HashMap<>(); + + int divisor = 1; + for ( final Map.Entry> entry : possibleValues.entrySet() ) { + final String alias = entry.getKey(); + final List validValues = entry.getValue(); + + final int idx = (iteration / divisor) % validValues.size(); + final Object obj = validValues.get(idx); + aliasMap.put(alias, obj); + + divisor *= validValues.size(); + } + + return aliasMap; + } + + public String toTreeString() { + final StringBuilder sb = new StringBuilder(); + toTreeString(tree, sb, 0); + return sb.toString(); + } + + private void toTreeString(final Tree tree, final StringBuilder sb, final int indentLevel) { + final String nodeName = tree.getText(); + for (int i=0; i < indentLevel; i++) { + sb.append(" "); + } + sb.append(nodeName); + sb.append("\n"); + + for (int i=0; i < tree.getChildCount(); i++) { + final Tree child = tree.getChild(i); + toTreeString(child, sb, indentLevel + 2); + } + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java new file mode 100644 index 0000000000..b198bc7ffb --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java @@ -0,0 +1,29 @@ +/* + * 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.hl7.query; + +import java.util.List; + +public interface QueryResult { + boolean isMatch(); + + List getLabels(); + + int getHitCount(); + + ResultHit nextHit(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java new file mode 100644 index 0000000000..ee97e5d94c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java @@ -0,0 +1,25 @@ +/* + * 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.hl7.query; + +import java.util.Map; + +public interface ResultHit { + Object getValue(String label); + + Map getSelectedValues(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java new file mode 100644 index 0000000000..36a181f539 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java @@ -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.hl7.query; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class Selection { + private final Evaluator evaluator; + private final String name; + + public Selection(final Evaluator evaluator, final String name) { + this.evaluator = evaluator; + this.name = name; + } + + public String getName() { + return name; + } + + public Evaluator getEvaluator() { + return evaluator; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java new file mode 100644 index 0000000000..fdd807ed3e --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java @@ -0,0 +1,24 @@ +/* + * 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.hl7.query.evaluator; + +public abstract class BooleanEvaluator implements Evaluator { + + public Class getType() { + return Boolean.class; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java new file mode 100644 index 0000000000..d86c30e06d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.query.evaluator; + +import java.util.Map; + +public interface Evaluator { + public static final String MESSAGE_KEY = "message"; + + T evaluate(Map objectMap); + + Class getType(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java new file mode 100644 index 0000000000..6afa9ed46c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator; + + +public abstract class IntegerEvaluator implements Evaluator { + + public Class getType() { + return Integer.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java new file mode 100644 index 0000000000..5f73649318 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java @@ -0,0 +1,25 @@ +/* + * 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.hl7.query.evaluator; + +public abstract class StringEvaluator implements Evaluator { + + public Class getType() { + return String.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java new file mode 100644 index 0000000000..a7fa1b7a29 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java @@ -0,0 +1,106 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public abstract class AbstractComparisonEvaluator extends BooleanEvaluator { + private final Evaluator lhs; + private final Evaluator rhs; + + public AbstractComparisonEvaluator(final Evaluator lhs, final Evaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + public final Boolean evaluate(final Map objectMap) { + final Object lhsValue = lhs.evaluate(objectMap); + if ( lhsValue == null ) { + return false; + } + + final Object rhsValue = rhs.evaluate(objectMap); + if ( rhsValue == null ) { + return false; + } + + return compareRaw(lhsValue, rhsValue); + } + + + private Boolean compareRaw(Object lhsValue, Object rhsValue) { + if ( lhsValue == null || rhsValue == null ) { + return false; + } + + if ( lhsValue instanceof HL7Field ) { + lhsValue = ((HL7Field) lhsValue).getValue(); + } + + if ( rhsValue instanceof HL7Field ) { + rhsValue = ((HL7Field) rhsValue).getValue(); + } + + if ( lhsValue == null || rhsValue == null ) { + return false; + } + + // both are collections, and compare(lhsValue, rhsValue) is false. + // this would be the case, for instance, if we compared field 1 of one segment to + // a field in another segment, and both fields had components. + if ( lhsValue instanceof Collection && rhsValue instanceof Collection ) { + return false; + } + + // if one side is a collection but the other is not, check if any element in that + // collection compares to the other element in a way that satisfies the condition. + // this would happen, for instance, if we check Segment1.Field5 = 'X' and field 5 repeats + // with a value "A~B~C~X~Y~Z"; in this case we do want to consider Field 5 = X as true. + if ( lhsValue instanceof Collection ) { + for ( final Object lhsObject : (Collection) lhsValue ) { + if ( compareRaw(lhsObject, rhsValue) ) { + return true; + } + } + + return false; + } + + if ( rhsValue instanceof Collection ) { + for ( final Object rhsObject : (Collection) rhsValue ) { + if ( compareRaw(rhsObject, lhsValue) ) { + return true; + } + } + + return false; + } + + if ( lhsValue != null && rhsValue != null && compare(lhsValue, rhsValue) ) { + return true; + } + + return false; + } + + protected abstract boolean compare(Object lhs, Object rhs); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java new file mode 100644 index 0000000000..2529c4914b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java @@ -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.hl7.query.evaluator.comparison; + +import java.util.regex.Pattern; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public abstract class AbstractNumericComparison extends AbstractComparisonEvaluator { + private static final Pattern NUMERIC_PATTERN = Pattern.compile("\\d+(\\.\\d+)?"); + + public AbstractNumericComparison(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected final boolean compare(final Object lhs, final Object rhs) { + final Double lhsDouble = toDouble(lhs); + if ( lhsDouble == null ) { + return false; + } + + final Double rhsDouble = toDouble(rhs); + if ( rhsDouble == null ) { + return false; + } + + return compareNumbers(lhsDouble, rhsDouble); + } + + private Double toDouble(final Object value) { + if ( value == null ) { + return null; + } + + if ( value instanceof Double ) { + return (Double) value; + } + if ( value instanceof Number ) { + return ((Number) value).doubleValue(); + } + + if ( value instanceof String ) { + if ( NUMERIC_PATTERN.matcher((String) value).matches() ) { + return Double.parseDouble((String) value); + } + } + + return null; + } + + protected abstract boolean compareNumbers(final Double lhs, final Double rhs); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java new file mode 100644 index 0000000000..7ee2f8707b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java @@ -0,0 +1,32 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class EqualsEvaluator extends AbstractComparisonEvaluator { + + public EqualsEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compare(final Object lhs, final Object rhs) { + return lhs != null && rhs != null && ((lhs == rhs) || (lhs.equals(rhs)) || lhs.toString().equals(rhs.toString())); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java new file mode 100644 index 0000000000..bf8596ecc9 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + + +public class GreaterThanEvaluator extends AbstractNumericComparison { + + public GreaterThanEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs > rhs; + } + + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java new file mode 100644 index 0000000000..69115a3855 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + + +public class GreaterThanOrEqualEvaluator extends AbstractNumericComparison { + + public GreaterThanOrEqualEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs >= rhs; + } + + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java new file mode 100644 index 0000000000..69d481e289 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java @@ -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.hl7.query.evaluator.comparison; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class IsNullEvaluator extends BooleanEvaluator { + private final Evaluator subjectEvaluator; + + public IsNullEvaluator(final Evaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public Boolean evaluate(final Map objectMap) { + Object subjectValue = subjectEvaluator.evaluate(objectMap); + if ( subjectValue == null ) { + return true; + } + + return isNull(subjectValue); + } + + private boolean isNull(Object subjectValue) { + if ( subjectValue == null ) { + return true; + } + + if ( subjectValue instanceof HL7Component ) { + subjectValue = ((HL7Component) subjectValue).getValue(); + } + + if ( subjectValue instanceof Collection ) { + final Collection collection = (Collection) subjectValue; + if ( collection.isEmpty() ) { + return true; + } + + for ( final Object obj : collection ) { + if ( !isNull(obj) ) { + return false; + } + } + + return true; + } + + return subjectValue == null; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java new file mode 100644 index 0000000000..891d5e4c3b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java @@ -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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class LessThanEvaluator extends AbstractNumericComparison { + public LessThanEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs < rhs; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java new file mode 100644 index 0000000000..c6fb0974ed --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java @@ -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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class LessThanOrEqualEvaluator extends AbstractNumericComparison { + public LessThanOrEqualEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs <= rhs; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java new file mode 100644 index 0000000000..b7c1ce28a0 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java @@ -0,0 +1,32 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class NotEqualsEvaluator extends AbstractComparisonEvaluator { + + public NotEqualsEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compare(final Object lhs, final Object rhs) { + return lhs != null && rhs != null && lhs != rhs && !lhs.equals(rhs) && !lhs.toString().equals(rhs.toString()); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java new file mode 100644 index 0000000000..58888d9aea --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.comparison; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; + +public class NotEvaluator extends BooleanEvaluator { + private final BooleanEvaluator subjectEvaluator; + + public NotEvaluator(final BooleanEvaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public Boolean evaluate(final Map objectMap) { + final Boolean subjectValue = subjectEvaluator.evaluate(objectMap); + return (subjectValue == null || Boolean.TRUE.equals(subjectValue)); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java new file mode 100644 index 0000000000..a764fef61c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java @@ -0,0 +1,65 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class NotNullEvaluator extends BooleanEvaluator { + private final Evaluator subjectEvaluator; + + public NotNullEvaluator(final Evaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public Boolean evaluate(final Map objectMap) { + Object subjectValue = subjectEvaluator.evaluate(objectMap); + if ( subjectValue == null ) { + return false; + } + + return isNotNull(subjectValue); + } + + private boolean isNotNull(Object subjectValue) { + if ( subjectValue instanceof HL7Component ) { + subjectValue = ((HL7Component) subjectValue).getValue(); + } + + if ( subjectValue instanceof Collection ) { + final Collection collection = (Collection) subjectValue; + if ( collection.isEmpty() ) { + return false; + } + + for ( final Object obj : collection ) { + if ( isNotNull(obj) ) { + return true; + } + } + + return false; + } + + return subjectValue != null; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java new file mode 100644 index 0000000000..c6ff6e479d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.literal; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; + +public class IntegerLiteralEvaluator extends IntegerEvaluator { + private final Integer value; + + public IntegerLiteralEvaluator(final Integer value) { + this.value = value; + } + + + @Override + public Integer evaluate(final Map objectMap) { + return value; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java new file mode 100644 index 0000000000..3b296117c9 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.literal; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.StringEvaluator; + +public class StringLiteralEvaluator extends StringEvaluator { + private final String value; + + public StringLiteralEvaluator(final String value) { + this.value = value; + } + + @Override + public String evaluate(final Map objectMap) { + return value; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java new file mode 100644 index 0000000000..21f596e3dc --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java @@ -0,0 +1,43 @@ +/* + * 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.hl7.query.evaluator.logic; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; + +public class AndEvaluator extends BooleanEvaluator { + private final BooleanEvaluator lhs; + private final BooleanEvaluator rhs; + + public AndEvaluator(final BooleanEvaluator lhs, final BooleanEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Boolean evaluate(final Map objectMap) { + final Boolean lhsValue = lhs.evaluate(objectMap); + if ( lhsValue == null || Boolean.FALSE.equals(lhsValue) ) { + return false; + } + + final Boolean rhsValue = rhs.evaluate(objectMap); + return (rhsValue != null && Boolean.TRUE.equals(rhsValue)); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java new file mode 100644 index 0000000000..d090946da0 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java @@ -0,0 +1,43 @@ +/* + * 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.hl7.query.evaluator.logic; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; + +public class OrEvaluator extends BooleanEvaluator { + private final BooleanEvaluator lhs; + private final BooleanEvaluator rhs; + + public OrEvaluator(final BooleanEvaluator lhs, final BooleanEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Boolean evaluate(final Map objectMap) { + final Boolean lhsValue = lhs.evaluate(objectMap); + if ( lhsValue != null && Boolean.TRUE.equals(lhsValue) ) { + return true; + } + + final Boolean rhsValue = rhs.evaluate(objectMap); + return (rhsValue != null && Boolean.TRUE.equals(rhsValue)); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java new file mode 100644 index 0000000000..6afb8d7ec1 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java @@ -0,0 +1,42 @@ +/* + * 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.hl7.query.evaluator.message; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.StringEvaluator; + +public class DeclaredReferenceEvaluator implements Evaluator { + private final StringEvaluator referenceNameEvaluator; + + public DeclaredReferenceEvaluator(final StringEvaluator referenceNameEvaluator) { + this.referenceNameEvaluator = referenceNameEvaluator; + } + + @Override + public Object evaluate(final Map objectMap) { + final String referenceName = referenceNameEvaluator.evaluate(objectMap); + return objectMap.get(referenceName); + } + + @Override + public Class getType() { + return Object.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java new file mode 100644 index 0000000000..c5fbf41c54 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java @@ -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.hl7.query.evaluator.message; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.model.HL7Segment; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; + +public class DotEvaluator implements Evaluator { + private final Evaluator lhs; + private final IntegerEvaluator rhs; + + public DotEvaluator(final Evaluator lhs, final IntegerEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Object evaluate(final Map objectMap) { + final Object lhsValue = this.lhs.evaluate(objectMap); + final Integer rhsValue = this.rhs.evaluate(objectMap); + + if ( lhsValue == null || rhsValue == null ) { + return null; + } + + final List results = new ArrayList<>(); + if ( lhsValue instanceof Collection ) { + final Collection lhsCollection = (Collection) lhsValue; + for ( final Object obj : lhsCollection ) { + final Object val = getValue(obj, rhsValue); + results.add(val); + } + } else { + final Object val = getValue(lhsValue, rhsValue); + return val; + } + + return results; + } + + private Object getValue(final Object lhsValue, final int rhsValue) { + final List list; + if ( lhsValue instanceof HL7Message ) { + list = ((HL7Message) lhsValue).getSegments(); + } else if ( lhsValue instanceof HL7Segment ) { + list = ((HL7Segment) lhsValue).getFields(); + } else if ( lhsValue instanceof HL7Component ) { + list = ((HL7Component) lhsValue).getComponents(); + } else { + return null; + } + + if ( rhsValue > list.size() ) { + return null; + } + + // convert from 0-based to 1-based + return list.get(rhsValue - 1); + } + + @Override + public Class getType() { + return Object.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java new file mode 100644 index 0000000000..869c2d0783 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java @@ -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.hl7.query.evaluator.message; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.model.HL7Segment; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; + +@SuppressWarnings("rawtypes") +public class FieldEvaluator implements Evaluator { + private final SegmentEvaluator segmentEvaluator; + private final IntegerEvaluator indexEvaluator; + + public FieldEvaluator(final SegmentEvaluator segmentEvaluator, final IntegerEvaluator indexEvaluator) { + this.segmentEvaluator = segmentEvaluator; + this.indexEvaluator = indexEvaluator; + } + + public List evaluate(final Map objectMap) { + final List segments = segmentEvaluator.evaluate(objectMap); + if ( segments == null ) { + return Collections.emptyList(); + } + + final Integer index = indexEvaluator.evaluate(objectMap); + if ( index == null ) { + return Collections.emptyList(); + } + + final List fields = new ArrayList<>(); + for ( final HL7Segment segment : segments ) { + final List segmentFields = segment.getFields(); + if ( segmentFields.size() <= index ) { + continue; + } + + fields.add(segmentFields.get(index)); + } + + return fields; + } + + public Class getType() { + return List.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java new file mode 100644 index 0000000000..5e08961954 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.hl7.query.evaluator.message; + +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class MessageEvaluator implements Evaluator { + + public HL7Message evaluate(final Map objectMap) { + return (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY); + } + + public Class getType() { + return HL7Message.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java new file mode 100644 index 0000000000..1b9782d0e6 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java @@ -0,0 +1,51 @@ +/* + * 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.hl7.query.evaluator.message; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.model.HL7Segment; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.StringEvaluator; + +@SuppressWarnings("rawtypes") +public class SegmentEvaluator implements Evaluator { + private final StringEvaluator segmentTypeEvaluator; + + public SegmentEvaluator(final StringEvaluator segmentTypeEvaluator) { + this.segmentTypeEvaluator = segmentTypeEvaluator; + } + + public List evaluate(final Map objectMap) { + final String segmentType = segmentTypeEvaluator.evaluate(objectMap); + if ( segmentType == null ) { + return Collections.emptyList(); + } + + final HL7Message message = (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY); + final List segments = message.getSegments(segmentType); + return (segments == null) ? Collections.emptyList() : segments; + } + + public Class getType() { + return List.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java new file mode 100644 index 0000000000..998f3bc87c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java @@ -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.hl7.query.exception; + +public class HL7QueryParsingException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public HL7QueryParsingException() { + super(); + } + + public HL7QueryParsingException(final Throwable cause) { + super(cause); + } + + public HL7QueryParsingException(final String message) { + super(message); + } + + public HL7QueryParsingException(final String message, final Throwable cause) { + super(message, cause); + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java new file mode 100644 index 0000000000..a6b36c8621 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java @@ -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.hl7.query.result; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.hl7.query.QueryResult; +import org.apache.nifi.hl7.query.ResultHit; +import org.apache.nifi.hl7.query.Selection; + +public class MissedResult implements QueryResult { + private final List selections; + + public MissedResult(final List selections) { + this.selections = selections; + } + + @Override + public List getLabels() { + final List labels = new ArrayList<>(); + for ( final Selection selection : selections ) { + labels.add(selection.getName()); + } + return labels; + } + + @Override + public boolean isMatch() { + return false; + } + + @Override + public ResultHit nextHit() { + return null; + } + + @Override + public int getHitCount() { + return 0; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java new file mode 100644 index 0000000000..fbc16cab4c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java @@ -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.hl7.query.result; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.hl7.query.QueryResult; +import org.apache.nifi.hl7.query.ResultHit; +import org.apache.nifi.hl7.query.Selection; + +public class StandardQueryResult implements QueryResult { + private final List selections; + private final Set> hits; + private final Iterator> hitIterator; + + public StandardQueryResult(final List selections, final Set> hits) { + this.selections = selections; + this.hits = hits; + + hitIterator = hits.iterator(); + } + + @Override + public boolean isMatch() { + return !hits.isEmpty(); + } + + @Override + public List getLabels() { + final List labels = new ArrayList<>(); + for ( final Selection selection : selections ) { + labels.add(selection.getName()); + } + return labels; + } + + @Override + public int getHitCount() { + return hits.size(); + } + + @Override + public ResultHit nextHit() { + if ( hitIterator.hasNext() ) { + return new StandardResultHit(hitIterator.next()); + } else { + return null; + } + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java new file mode 100644 index 0000000000..944e998173 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java @@ -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.hl7.query.result; + +import java.util.Collections; +import java.util.Map; + +import org.apache.nifi.hl7.query.ResultHit; + +public class StandardResultHit implements ResultHit { + private final Map values; + + public StandardResultHit(final Map values) { + this.values = values; + } + + @Override + public Object getValue(final String label) { + return values.get(label); + } + + @Override + public Map getSelectedValues() { + return Collections.unmodifiableMap(values); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java b/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java new file mode 100644 index 0000000000..fbe4a8d285 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java @@ -0,0 +1,352 @@ +/* + * 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.hl7.query; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.hapi.HapiMessage; +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.query.HL7Query; +import org.apache.nifi.hl7.query.QueryResult; +import org.junit.Test; + +import ca.uhn.hl7v2.DefaultHapiContext; +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.HapiContext; +import ca.uhn.hl7v2.model.Message; +import ca.uhn.hl7v2.parser.PipeParser; +import ca.uhn.hl7v2.validation.impl.ValidationContextFactory; + +@SuppressWarnings("resource") +public class TestHL7Query { + + @Test + public void testAssignAliases() { + final LinkedHashMap> possibleValueMap = new LinkedHashMap<>(); + + final List valuesA = new ArrayList<>(); + valuesA.add("a"); + valuesA.add("b"); + valuesA.add("c"); + + final List valuesB = new ArrayList<>(); + valuesB.add("d"); + + final List valuesC = new ArrayList<>(); + valuesC.add("e"); + valuesC.add("f"); + + final List valuesD = new ArrayList<>(); + valuesD.add("g"); + valuesD.add("h"); + + possibleValueMap.put("A", valuesA); + possibleValueMap.put("B", valuesB); + possibleValueMap.put("C", valuesC); + possibleValueMap.put("D", valuesD); + + for (int i=0; i < valuesA.size() * valuesB.size() * valuesC.size() * valuesD.size(); i++) { + System.out.println(i + " : " + HL7Query.assignAliases(possibleValueMap, i)); + } + + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 0), "a", "d", "e", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 1), "b", "d", "e", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 2), "c", "d", "e", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 3), "a", "d", "f", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 4), "b", "d", "f", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 5), "c", "d", "f", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 6), "a", "d", "e", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 7), "b", "d", "e", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 8), "c", "d", "e", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 9), "a", "d", "f", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 10), "b", "d", "f", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 11), "c", "d", "f", "h"); + } + + private void verifyAssignments(final Map map, final String a, final String b, final String c, final String d) { + assertEquals(a, map.get("A")); + assertEquals(b, map.get("B")); + assertEquals(c, map.get("C")); + assertEquals(d, map.get("D")); + } + + @Test + public void testSelectMessage() throws HL7Exception, IOException { + final HL7Query query = HL7Query.compile("SELECT MESSAGE"); + final HL7Message msg = createMessage(new File("src/test/resources/vaers-message-long")); + final QueryResult result = query.evaluate(msg); + assertTrue(result.isMatch()); + final List labels = result.getLabels(); + assertEquals(1, labels.size()); + assertEquals("MESSAGE", labels.get(0)); + + assertEquals(1, result.getHitCount()); + assertEquals(msg, result.nextHit().getValue("MESSAGE")); + } + + @Test + @SuppressWarnings({ "unchecked", "rawtypes" }) + public void testSelectField() throws HL7Exception, IOException { + final HL7Query query = HL7Query.compile("SELECT PID.5"); + final HL7Message msg = createMessage(new File("src/test/resources/unsolicited-vaccine-update-short")); + final QueryResult result = query.evaluate(msg); + assertTrue(result.isMatch()); + final List labels = result.getLabels(); + assertEquals(1, labels.size()); + assertEquals(1, result.getHitCount()); + + final Object names = result.nextHit().getValue("PID.5"); + assertTrue(names instanceof List); + final List nameList = (List) names; + assertEquals(1, nameList.size()); + final HL7Field nameField = (HL7Field) nameList.get(0); + assertEquals("KENNEDY^JOHN^FITZGERALD^JR", nameField.getValue()); + } + + @Test + public void testSelectAbnormalTestResult() throws HL7Exception, IOException { + final String query = "DECLARE result AS REQUIRED OBX SELECT result WHERE result.7 != 'N' AND result.1 = 1"; + + final HL7Query hl7Query = HL7Query.compile(query); + final QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/vaers-message-long"))); + assertFalse( result.isMatch() ); + } + + + @Test + public void testFieldEqualsString() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'L'"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'H'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testLessThan() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 < 600"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 < 59"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testCompareTwoFields() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 < result.6.2"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE NOT(result.4 > result.6.3)"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testLessThanOrEqual() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 <= 59"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 <= 600"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 <= 58"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testGreaterThanOrEqual() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 >= 59"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 >= 6"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 >= 580"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testGreaterThan() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 > 58"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 > 6"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 > 580"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + + @Test + public void testDistinctValuesReturned() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result1 AS REQUIRED OBX, result2 AS REQUIRED OBX SELECT MESSAGE WHERE result1.7 = 'L' OR result2.7 != 'H'"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + assertEquals(1, result.getHitCount()); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT result WHERE result.1 = 1"); + HL7Message msg = createMessage(new File("src/test/resources/vaers-message-long")); + result = hl7Query.evaluate(msg); + assertTrue( result.isMatch() ); + assertEquals(9, result.getHitCount()); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT result WHERE result.1 = 1 AND result.3.1.1 = '30961-7'"); + result = hl7Query.evaluate(msg); + assertTrue( result.isMatch() ); + assertEquals(1, result.getHitCount()); + + } + + @Test + public void testAndWithParens() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'L' AND result.3.1 = 'GLU'"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'L' AND result.3.1 = 'GLU'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'H' AND result.3.1 = 'GLU'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'H' AND result.3.1 = 'GLU'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE (result.7 = 'H') AND (result.3.1 = 'GLU')"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE ((result.7 = 'H') AND (result.3.1 = 'GLU'))"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE (( ((result.7 = 'H')) AND ( ((result.3.1 = 'GLU')) )))"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + } + + + @Test + public void testIsNull() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.999 IS NULL"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.1 IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE ZZZ IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE OBX IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE NK1.1 = '1' AND NK1.8 IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/unsolicited-vaccine-update-long"))); + assertTrue( result.isMatch() ); + } + + + @Test + public void testNotNull() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.999 NOT NULL"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.1 NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE ZZZ NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE OBX NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE NK1.1 = '1' AND NK1.33 NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/unsolicited-vaccine-update-long"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE NK1.1 = 1 AND NK1.33 NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/unsolicited-vaccine-update-long"))); + assertTrue( result.isMatch() ); + } + + private HL7Message createMessage(final File file) throws HL7Exception, IOException { + final byte[] bytes = Files.readAllBytes(file.toPath()); + final String msgText = new String(bytes, "UTF-8"); + + final HapiContext hapiContext = new DefaultHapiContext(); + hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + + final PipeParser parser = hapiContext.getPipeParser(); + final Message message = parser.parse(msgText); + return new HapiMessage(message); + } + + @Test + @SuppressWarnings("unused") + public void createMessage() throws IOException, HL7Exception { + final byte[] bytes = Files.readAllBytes(Paths.get("src/test/resources/vaers-message-long")); + final String msgText = new String(bytes, "UTF-8"); + + final HapiContext hapiContext = new DefaultHapiContext(); + hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + + final PipeParser parser = hapiContext.getPipeParser(); + final Message message = parser.parse(msgText); + + final HL7Message hl7Msg = new HapiMessage(message); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia new file mode 100644 index 0000000000..dc44b89ac2 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia @@ -0,0 +1,5 @@ +MSH|^~\&|CERNER||PriorityHealth||||ORU^R01|Q479004375T431430612|P|2.3| +PID|||001677980||SMITH^CURTIS||19680219|M||||||||||929645156318|123456789| +PD1||||1234567890^LAST^FIRST^M^^^^^NPI| +OBR|1|341856649^HNAM_ORDERID|000002006326002362|648088^Basic Metabolic Panel|||20061122151600|||||||||1620^Hooker^Robert^L||||||20061122154733|||F|||||||||||20061122140000| +OBX|1|NM|GLU^Glucose Lvl|159|mg/dL|65-99^65^99|H|||F|||20061122154733| \ No newline at end of file diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia new file mode 100644 index 0000000000..02e8967296 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia @@ -0,0 +1,5 @@ +MSH|^~\&|CERNER||PriorityHealth||||ORU^R01|Q479004375T431430612|P|2.3| +PID|||001677980||SMITH^CURTIS||19680219|M||||||||||929645156318|123456789| +PD1||||1234567890^LAST^FIRST^M^^^^^NPI| +OBR|1|341856649^HNAM_ORDERID|000002006326002362|648088^Basic Metabolic Panel|||20061122151600|||||||||1620^Hooker^Robert^L||||||20061122154733|||F|||||||||||20061122140000| +OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20061122154733| \ No newline at end of file diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel new file mode 100644 index 0000000000..c62fc45a32 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel @@ -0,0 +1,23 @@ +MSH|^~\&|Lab1^1234^CLIA|^1234^CLIA|ELR^2.16.840.1.113883.19.3.2^ISO|SPH^2.16.840.1.113883.19.3.2^ISO|20110410140502-0500||ORU^R01^ORU_R01|1234567890|P^T|2.5.1|||NE|NE|USA||||USELR1.0^^2.16.840.1.114222.4.10.3^ISO +SFT|1|Level Seven Healthcare Software, Inc.^L^^^^&2.16.840.1.113883.19.4.6^ISO^XX^^^1234|1.2|An Lab System|56734||20080817 +PID|1||36363636^^^MPI&2.16.840.1.113883.19.3.2.1&ISO^MR^A&2.16.840.1.113883.19.3.2.1&ISO~444333333^^^&2.16.840.1.113883.4.1^IS O^SS||Everyman^Adam^A^^^^L^^^^^^^BS|Mum^Martha^M^^^^M|19800602|M||2106-3^White^CDCREC^^^^04/24/2007|2222 Home Street^^Ann Arbor^MI^99999^USA^H||^PRN^PH^^1^555^5552004|^WPN^PH^^1^955^5551009|eng^English^ISO6392^^^^3/29/2007|M^Married^HL70002^^^^2.5.1||||||N^Not Hispanic or Latino^HL70189^^^^2.5.1||||||||N|||200808151000-0700| Reliable^2.16.840.1.113883.19.3.1^ISO +ORC|RE|23456^EHR^2.16.840.1.113883.19.3.2.3^ISO|9700123^Lab^2.16.840.1.113883.19.3.1.6^ISO|||||||||1234^Admit^Alan^A^III^Dr^^^&2.16.840.1.113883.19.4.6^ISO^L^^^EI^&2.16.840.1.113883.19.4.6^ISO^^^^^^^^MD||^WPN^PH^^1^555^5551005|||||||Level Seven Healthcare, Inc.^L^^^^&2.16.840.1.113883.19.4.6^ISO^XX^^^1234|1005 Healthcare Drive^^Ann Arbor^MI^99999^USA^B|^WPN^PH^^1^555^5553001|4444 Healthcare Drive^Suite 123^Ann Arbor^MI^99999^USA^B +OBR|1|23456^EHR^2.16.840.1.113883.19.3.2.3^ISO|9700123^Lab^2.16.840.1.113883.19.3.1.6^ISO|24323-8^Comprehensive metabolic 2000 panel in Serum or Plasma^LN^3436442^Metaboloic Panel 2000, Comprehensive^99USI|||201104101130-0500||||||angina|||1234^Admit^Alan^A^III^Dr^^^&2.16.840.1.113883.19.4.6^ISO^L^^^EI^&2.16.840.1.113883.19.4.6^ISO^^^^^^^^MD|^WPN^PH^^1^555^5551005|||||201104101405-0500|||F||||||413^Angina pectoris^I9CDX^^^^07/09/2008|1235&Slide&Stan&S&&Dr&MD&&DOC&2.16.840.1.113883.19.4.6&ISO +OBX|1|NM|17861-6^Calcium [Mass/volume] in Serum or Plasma^LN||27.3|mg/dL^milligrams per deciliter^UCUM|8.7-10.7|HH|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|2|NM|3094-0^Urea nitrogen [Mass/volume] in Serum of Plasma^LN||15|mg/dL^milligrams per deciliter^UCUM|6 to 23|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|3|NM|2160-0^Creatinine [Mass/volume] in Serum or Plasma^LN||1.8|mg/dL^milligrams per deciliter^UCUM|0.7 to 1.2|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|4|NM|3097-3^Urea nitrogen/Creatinine [Mass ratio] in Serum or Plasma^LN||15||6 to 25|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|5|NM|2885-2^Protein [Mass/volume] in Serum or Plasma^LN||8.9|gm/dL^grams per deciliter^UCUM|6.3 to 8.2|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|6|NM|1751-7^Albumin [Mass/volume] in Serum or Plasma^LN||5.7|gm/dL^grams per deciliter^UCUM|3.5 to 5.0|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|7|NM|2336-6^Globulin [Mass/volume] in Serum or Plasma^LN||4.7|gm/dL^grams per deciliter^UCUM|2.2 to 4.2|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|8|NM|1759-0^Albumin/Globulin [Mass ratio] in Serum or Plasma^LN||1.7||0.8 to 2.0|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|9|NM|1975-2^Bilirubin.total [Mass/volume] in Serum or Plasma^LN||0.7|mg/dL^milligrams per deciliter^UCUM|0.3 to 1.9|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|10|NM|2345-7^Glucose [Mass/volume] in Serum or Plasma^LN||55|mg/dL^milligrams per deciliter^UCUM|60 to 109|L|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|11|NM|6768-6^Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma^LN||64|U/L^units per liter^UCUM|32 to 110|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|12|NM|1920-8^Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma^LN||6|U/L^units per liter^UCUM|6 to 18|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|13|NM|1742-6^Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma^LN||10|U/L^units per liter^UCUM|5 to 35|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|14|NM|2951-2^Sodium [Moles/volume] in Serum or Plasma^LN||140|mmol/L^millimoles per liter^UCUM|137 to 147|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|15|NM|2823-3^Potassium [Moles/volume] in Serum or Plasma^LN||4.5|mmol/L^millimoles per liter^UCUM|3.4 to 5.3|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|16|NM|2075-0^Chloride [Moles/volume] in Serum or Plasma^LN||99|mmol/L^millimoles per liter^UCUM|99 to 108|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +OBX|17|NM|2028-9^Carbon dioxide, total [Moles/volume] in Serum or Plasma^LN||27|mmol/L^millimoles per liter^UCUM|22 to 29|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI +SPM|1|23456&EHR&2.16.840.1.113883.19.3.2.3&ISO^9700122&Lab&2.16.840.1.113883.19.3.1.6&ISO||119364003^Serum specimen^SCT^^^^20080131|||||||||||||201104101130-0500|201104101130-0500 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long new file mode 100644 index 0000000000..8edd3fd82e --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long @@ -0,0 +1,16 @@ +MSH|^~\&||MA0000||GA0000|19970901||VXU^V04|19970522MA53|T|2.3.1|||AL +PID|||1234^^^^SR^~1234-12^^^^LR^~3872^^^^MR~221345671^^^^SS^~430078856^^^^MA^ ||KENNEDY^JOHN^FITZGERALD^JR^^^L|BOUVIER^^^^^^M|19900607|M|KENNEDY^BABY BOY^^^^^^ B|W^WHITE^NY8 RACE CODES^W^WHITE^HL70005|123 MAIN ST^APT 3B^LEXINGTON^MA^00210^ ^M^MSA CODE^MA034~345 ELM ST^^BOSTON^MA^00314^^BLD~^^^^^^BR^^MA002| |(617) 555-1212 ^PRN^PH^^^617^5551212^^||EN^ENGLISH^HL70296^^^|||||||WN^NOT HISPANIC^LOCAL CODE SET^NH^NOT OF HISPANIC ORIGIN^HL70189|CHILDREN=S HOSPITAL +PD1|||CHILDREN=S HOSPITAL^^1234^^^^XX~LEXINGTON CLINIC^^1234A^^^^FI|12345^CARE^ PRIMARY^^^DR^MD^^^L^^^DN|||||||03^REMINDER/RECALL - NO CALLS^HL70215|Y +NK1|1|KENNEDY^JACQUELINE^LEE|32^MOTHER^HL70063||||||||||||||||||||||||||||||898666725^^^^SS +NK1|2|KENNEDY^JOHN^FITZGERALD|33^FATHER^HL70063||||||||||||||||||||||||||||||822546618^^^^SS +PV1||R|||||||||||||||A|||V02^19900607~H02^19900607 +RXA|0|1|19900607|19900607|08^HEPB-PEDIATRIC/ADOLESCENT^CVX^90744^HEPB-PEDATRIC/ADOLESCENT^CPT|.5|ML^^ISO+||03^HISTORICAL INFORMATION - FROM PARENT=S WRITTEN RECORD^NIP0001|^JONES^LISA|^^^CHILDREN=S HOSPITAL||5|MCG^^ISO+|MRK12345| 199206|MSD^MERCK^MVX +RXA|0|4|19910907|19910907|50^DTAP-HIB^CVX^90721^DTAP-HIB^CPT|.5|ML^^ISO+||00^NEW IMMUNIZATION RECORD^NIP0001|1234567890^SMITH^SALLY^S^^^^^^^^^VEI~1234567891 ^O=BRIAN^ROBERT^A^^DR^MD^^^^^^OEI|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^ BOSTON^MA||||W46932777|199208|PMC^PASTEUR MERIEUX CONNAUGHT^MVX|||CP|A| 19910907120030 +RXR|IM^INTRAMUSCULAR^HL70162|LA^LEFT ARM^HL70163 +RXA|0|1|19910907|19910907|03^MMR^CVX|.5|ML^^ISO+|||1234567890^SMITH^SALLY^S^^^^^^^^^VEI~1234567891^O=BRIAN^ROBERT^A^^DR^MD^^^^^^OEI|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^BOSTON^MA||||W2348796456|19920731|MSD^MERCK^MVX +RXR|SC^SUBCUTANEOUS^HL70162|LA^LEFT ARM^HL70163 +RXA|0|5|19950520|19950520|20^DTAP^CVX|.5|ML^^ISO+|||1234567891^O=BRIAN^ROBERT^A^^DR|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^BOSTON^MA||||W22532806|19950705|PMC^ PASTEUR MERIEUX CONNAUGHT^MVX +RXR|IM^INTRAMUSCULAR^HL70162|LA^LEFT ARM^HL70163 +NTE|PATIENT DEVELOPED HIGH FEVER APPROX 3 HRS AFTER VACCINE INJECTION +RXA|0|2|19950520|19950520|03^MMR^CVX|.5|ML^^ISO+|||1234567891^O=BRIAN^ROBERT^A^^DR|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^BOSTON^MA||||W2341234567|19950630| MSD^MERCK^MVX +RXR|SC^SUBCUTANEOUS^HL70162|LA^LEFT ARM^HL70163 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short new file mode 100644 index 0000000000..c3e7cf0ff2 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short @@ -0,0 +1,4 @@ +MSH|^~\&|||||||VXU^V04|19970522MA53|P|2.3.1 +PID|||221345671^^^^SS||KENNEDY^JOHN^FITZGERALD^JR|BOUVIER^^^^^^M|19900607|M|||^^^^MA^^^BLD +NK1|1|KENNEDY^JACQUELINE^LEE|32^MOTHER^HL70063 +RXA|0|1|19900607|19900607|08^HEPB-PEDIATRIC/ADOLESCENT^CVX|.5|ML^^ISO+||||||||MRK12345||MSD^MERCK^MVX diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query new file mode 100644 index 0000000000..1bd47b285e --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query @@ -0,0 +1,3 @@ +MSH|^~\&||GA0000||MA0000|199705221605||VXQ^V01|19970522GA40|T|2.3.1|||AL +QRD|199705221605|R|I|19970522GA05|||25^RD|^KENNEDY^JOHN^FITZGERALD^JR|VXI^VACCINE INFORMATION^HL70048|^SIIS +QRF|MA0000||||256946789~19900607~MA~MA99999999~88888888~KENNEDY^JACQUELINE^LEE~BOUVIER~898666725~KENNEDY^JOHN^FITZGERALD~822546618 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long new file mode 100644 index 0000000000..888b04cc0f --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long @@ -0,0 +1,60 @@ +MSH|^~\&||GA0000||VAERS PROCESSOR|20010331605||ORU^R01|20010422GA03|T|2.3.1|||AL| +PID|||1234^^^^SR~1234-12^^^^LR~00725^^^^MR||Doe^John^Fitzgerald^JR^^^L||20001007|M||2106-3^White^HL70005|123 Peachtree St^APT 3B^Atlanta^GA^30210^^M^^GA067||(678) 555-1212^^PRN| +NK1|1|Jones^Jane^Lee^^RN|VAB^Vaccine administered by (Name)^HL70063| +NK1|2|Jones^Jane^Lee^^RN|FVP^Form completed by (Name)-Vaccine provider^HL70063|101 Main Street^^Atlanta^GA^38765^^O^^GA121||(404) 554-9097^^WPN| +ORC|CN|||||||||||1234567^Welby^Marcus^J^Jr^Dr.^MD^L|||||||||Peachtree Clinic|101 Main Street^^Atlanta^GA^38765^^O^^GA121|(404) 554-9097^^WPN|101 Main Street^^Atlanta^GA^38765^^O^^GA121| +OBR|1|||^CDC VAERS-1 (FDA) Report|||20010316| +OBX|1|NM|21612-7^Reported Patient Age^LN||05|mo^month^ANSI| +OBX|1|TS|30947-6^Date form completed^LN||20010316| +OBX|2|FT|30948-4^Vaccination adverse events and treatment, if any^LN|1|fever of 106F, with vomiting, seizures, persistent crying lasting over 3 hours, loss of appetite| +OBX|3|CE|30949-2^Vaccination adverse event outcome^LN|1|E^required emergency room/doctor visit^NIP005| +OBX|4|CE|30949-2^Vaccination adverse event outcome^LN|1|H^required hospitalization^NIP005| +OBX|5|NM|30950-0^Number of days hospitalized due to vaccination adverse event^LN|1|02|d^day^ANSI| +OBX|6|CE|30951-8^Patient recovered^LN||Y^Yes^ HL70239| +OBX|7|TS|30952-6^Date of vaccination^LN||20010216| +OBX|8|TS|30953-4^Adverse event onset date and time^LN||200102180900| +OBX|9|FT|30954-2^Relevant diagnostic tests/lab data^LN||Electrolytes, CBC, Blood culture| +OBR|2|||30955-9^All vaccines given on date listed in #10^LN| +OBX|1|CE30955-9&30956-7^Vaccine type^LN|1|08^HepB-Adolescent/pediatric^CVX| +OBX|2|CE|30955-9&30957-5^Manufacturer^LN|1|MSD^Merck^MVX| +OBX|3|ST|30955-9&30959-1^Lot number^LN|1|MRK12345| +OBX|4|CE|30955-9&30958-3^ Route^LN|1|IM^Intramuscular ^HL70162| +OBX|5|CE|30955-9&31034-2^Site^LN|1|LA^Left arm^ HL70163| +OBX|6|NM|30955-9&30960-9^Number of previous doses^LN|1|01I +OBX|7|CE|CE|30955-9&30956-7^Vaccine type^LN|2|50^DTaP-Hib^CVX| +OBX|8|CE|30955-9&30957-5^ Manufacturer^LN|2|WAL^Wyeth_Ayerst^MVX| +OBX|9|ST|30955-9&30959-1^Lot number^LN|2|W46932777| +OBX|10|CE|30955-9&30958-3^ Route^LN|2|IM^Intramuscular^HL70162| +OBX|11|CE|30955-9&31034-2^Site^LN|2|LA^Left arm^HL70163| +OBX|12|NM|30955-9&30960-9^Number of previous doses^LN|2|01| +OBR|3|||30961-7^Any other vaccinations within 4 weeks prior to the date listed in #10| +OBX|1|CE|30961-7&30956-7^Vaccine type^LN|1|10^IPV^CVX| +OBX|2|CE|30961-7&30957-5^Manufacturer^LN|1|PMC^Aventis Pasteur ^MVX| +OBX|3|ST|30961-7&30959-1^Lot number^LN|1|PMC123456| +OBX|4|CE|30961-7&30958-3^Route^LN|1|SC^Subcutaneaous^HL70162| +OBX|5|CE|30961-7&31034-2^Site^LN|1|LA^Left arm^HL70163| +OBX|6|NM|30961-7&30960-9^Number of previous doses^LN|1|01| +OBX|7|TS|30961-7&31035-9^date given^LN|1|20001216| +OBX|8|CE|30962-^Vaccinated at^LN||PVT^Private doctor�s office/hospital^NIP009| +OBX|9|CE|30963-3^Vaccine purchased with^LN||PBF^Public funds^NIP008| +OBX|10|FT|30964-1^Other medications^LN||None| +OBX|11|FT|30965-8^Illness at time of vaccination (specify)^LN||None| +OBX|12|FT|30966-6^Pre-existing physician diagnosed allergies, birth defects, medical conditions^LN||Past conditions convulsions| +OBX|13|CE|30967-4^Was adverse event reported previously^LN||N^no^NIP009| +OBR|4||30968-2^Adverse event following prior vaccination in patient^LN| +OBX|1|TX|30968-2&30971-6^Adverse event^LN||None| +OBR|5||30969-0^Adverse event following prior vaccination in brother^LN| +OBX|1|TX||30969-0&30971-6^Adverse event^LN||vomiting, fever, otitis media| +OBX|2|NM||30969-0&30972-4^Onset age^LN||04|mo^month^ANSI| +OBX|3|CE||30969-0&30956-7^Vaccine Type ^LN||10^IPV^CVX| +OBX|4|NM||30969-0&30973-2^Dose number in series^LN||02| +OBR|6|||30970-8^Adverse event following prior vaccination in sister^LN| +OBX|1|TX|30970-8&30971-6^Adverse event^LN||None| +OBR|7||^For children 5 and under| +OBX|1|NM|8339-4^Body weight at birth^LN||82|oz^ounces^ANSI| +OBX|2|NM|30974-0^Number of brothers and sisters^LN||2| +OBR|8|||^Only for reports submitted by manufacturer/immunization project| +OBX|1|ST|30975-7^Mfr./Imm. Proj. report no.^LN||12345678| +OBX|2|TS|30976-5^Date received by manufacturer/immunization project^LN||12345678| +OBX|3|CE|30977-3^15 day report^LN||N^No^HL70136| +OBX|4|CE|30978-1^Report type^LN||IN^Initial^NIP010| diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml new file mode 100644 index 0000000000..391206e25a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-nar/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-hl7-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-hl7-nar + nar + + + + org.apache.nifi + nifi-hl7-processors + 0.1.0-incubating-SNAPSHOT + + + + diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/.gitignore b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/.gitignore new file mode 100644 index 0000000000..b83d22266a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml new file mode 100644 index 0000000000..2a0c90905b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml @@ -0,0 +1,106 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-hl7-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-hl7-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + + org.apache.nifi + nifi-hl7-query-language + 0.1.0-incubating-SNAPSHOT + + + + ca.uhn.hapi + hapi-base + 2.2 + + + ca.uhn.hapi + hapi-structures-v21 + 2.2 + + + ca.uhn.hapi + hapi-structures-v22 + 2.2 + + + ca.uhn.hapi + hapi-structures-v23 + 2.2 + + + ca.uhn.hapi + hapi-structures-v231 + 2.2 + + + ca.uhn.hapi + hapi-structures-v24 + 2.2 + + + ca.uhn.hapi + hapi-structures-v25 + 2.2 + + + ca.uhn.hapi + hapi-structures-v251 + 2.2 + + + ca.uhn.hapi + hapi-structures-v26 + 2.2 + + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java new file mode 100644 index 0000000000..e4a0d53646 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java @@ -0,0 +1,247 @@ +/* + * 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.hl7; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; +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.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.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.stream.io.StreamUtils; + +import ca.uhn.hl7v2.DefaultHapiContext; +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.HapiContext; +import ca.uhn.hl7v2.model.Composite; +import ca.uhn.hl7v2.model.Group; +import ca.uhn.hl7v2.model.Message; +import ca.uhn.hl7v2.model.Primitive; +import ca.uhn.hl7v2.model.Segment; +import ca.uhn.hl7v2.model.Structure; +import ca.uhn.hl7v2.model.Type; +import ca.uhn.hl7v2.model.Varies; +import ca.uhn.hl7v2.parser.PipeParser; +import ca.uhn.hl7v2.validation.impl.ValidationContextFactory; + + +@SideEffectFree +@SupportsBatching +@Tags({"HL7", "health level 7", "healthcare", "extract", "attributes"}) +@CapabilityDescription("Extracts information from an HL7 (Health Level 7) formatted FlowFile and adds the information as FlowFile Attributes. " + + "The attributes are named as . If the segment is repeating, the naming will be " + + " . For example, we may have an attribute named \"MHS.12\" with " + + "a value of \"2.1\" and an attribute named \"OBX_11.3\" with a value of \"93000^CPT4\".") +public class ExtractHL7Attributes extends AbstractProcessor { + public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() + .name("Character Encoding") + .description("The Character Encoding that is used to encode the HL7 data") + .required(true) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile is routed to this relationship if it is properly parsed as HL7 and its attributes extracted") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFile is routed to this relationship if it cannot be mapped to FlowFile Attributes. This would happen if the FlowFile does not contain valid HL7 data") + .build(); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(); + properties.add(CHARACTER_SET); + return properties; + } + + @Override + public Set getRelationships() { + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + return relationships; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).evaluateAttributeExpressions(flowFile).getValue()); + + final byte[] buffer = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, buffer); + } + }); + + @SuppressWarnings("resource") + final HapiContext hapiContext = new DefaultHapiContext(); + hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + + final PipeParser parser = hapiContext.getPipeParser(); + final String hl7Text = new String(buffer, charset); + final Message message; + try { + message = parser.parse(hl7Text); + final Group group = message.getParent(); + + final Map attributes = new HashMap<>(); + extractAttributes(group, attributes); + flowFile = session.putAllAttributes(flowFile, attributes); + getLogger().info("Successfully extracted {} attributes for {}; routing to success", new Object[] {attributes.size(), flowFile}); + getLogger().debug("Added the following attributes for {}: {}", new Object[] {flowFile, attributes}); + session.transfer(flowFile, REL_SUCCESS); + } catch (final HL7Exception e) { + getLogger().error("Failed to extract attributes from {} due to {}", new Object[] {flowFile, e}); + session.transfer(flowFile, REL_FAILURE); + return; + } + } + + private void extractAttributes(final Group group, final Map attributes) throws HL7Exception { + extractAttributes(group, attributes, new HashMap()); + } + + private void extractAttributes(final Group group, final Map attributes, final Map segmentCounts) throws HL7Exception { + if ( group.isEmpty() ) { + return; + } + + final String[] structureNames = group.getNames(); + for ( final String structName : structureNames ) { + final Structure[] subStructures = group.getAll(structName); + + if ( group.isGroup(structName) ) { + for ( final Structure subStructure : subStructures ) { + final Group subGroup = (Group) subStructure; + extractAttributes(subGroup, attributes, segmentCounts); + } + } else { + for ( final Structure structure : subStructures ) { + final Segment segment = (Segment) structure ; + + final String segmentName = segment.getName(); + Integer segmentNum = segmentCounts.get(segmentName); + if (segmentNum == null) { + segmentNum = 1; + segmentCounts.put(segmentName, 1); + } else { + segmentNum++; + segmentCounts.put(segmentName, segmentNum); + } + + final boolean segmentRepeating = segment.getParent().isRepeating(segment.getName()); + final boolean parentRepeating = (segment.getParent().getParent() != segment.getParent() && segment.getParent().getParent().isRepeating(segment.getParent().getName())); + final boolean useSegmentIndex = segmentRepeating || parentRepeating; + + final Map attributeMap = getAttributes(segment, useSegmentIndex ? segmentNum : null); + attributes.putAll(attributeMap); + } + } + } + } + + + private Map getAttributes(final Segment segment, final Integer segmentNum) throws HL7Exception { + final Map attributes = new HashMap<>(); + + for (int i=1; i <= segment.numFields(); i++) { + final String fieldName = segment.getName() + (segmentNum == null ? "" : "_" + segmentNum) + "." + i; + final Type[] types = segment.getField(i); + final StringBuilder sb = new StringBuilder(); + for ( final Type type : types ) { + final String typeValue = getValue(type); + if ( !typeValue.isEmpty() ) { + sb.append(typeValue).append("^"); + } + } + + if ( sb.length() == 0 ) { + continue; + } + String typeVal = sb.toString(); + if ( typeVal.endsWith("^") ) { + typeVal = typeVal.substring(0, typeVal.length() - 1); + } + + attributes.put(fieldName, typeVal); + } + + return attributes; + } + + + private String getValue(final Type type) { + if ( type == null ) { + return ""; + } + + if ( type instanceof Primitive ) { + final String value = ((Primitive) type).getValue(); + return value == null ? "" : value; + } else if ( type instanceof Composite ) { + final StringBuilder sb = new StringBuilder(); + final Composite composite = (Composite) type; + for ( final Type component : composite.getComponents() ) { + final String componentValue = getValue(component); + if ( !componentValue.isEmpty() ) { + sb.append(componentValue).append("^"); + } + } + + final String value = sb.toString(); + if ( value.endsWith("^") ) { + return value.substring(0, value.length() - 1); + } + + return value; + } else if ( type instanceof Varies ) { + final Varies varies = (Varies) type; + return getValue(varies.getData()); + } + + return ""; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java new file mode 100644 index 0000000000..c8c4176a0b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java @@ -0,0 +1,217 @@ +/* + * 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.hl7; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; +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.annotation.behavior.DynamicProperties; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +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.hl7.hapi.HapiMessage; +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.query.HL7Query; +import org.apache.nifi.hl7.query.QueryResult; +import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.stream.io.StreamUtils; + +import ca.uhn.hl7v2.DefaultHapiContext; +import ca.uhn.hl7v2.HapiContext; +import ca.uhn.hl7v2.model.Message; +import ca.uhn.hl7v2.parser.PipeParser; +import ca.uhn.hl7v2.validation.impl.ValidationContextFactory; + +@EventDriven +@SideEffectFree +@SupportsBatching +@Tags({"HL7", "healthcare", "route", "Health Level 7"}) +@DynamicProperties({@DynamicProperty(name="Name of a Relationship", value="An HL7 Query Language query", description="If a FlowFile matches the query, it will be routed to a relationship with the name of the property")}) +@WritesAttributes({@WritesAttribute(attribute="RouteHL7.Route", description="The name of the relationship to which the FlowFile was routed")}) +@CapabilityDescription("Routes incoming HL7 data according to user-defined queries. To add a query, add a new property to the processor." + + " The name of the property will become a new relationship for the processor, and the value is an HL7 Query Language query. If" + + " a FlowFile matches the query, a copy of the FlowFile will be routed to the associated relationship.") +public class RouteHL7 extends AbstractProcessor { + public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() + .name("Character Encoding") + .description("The Character Encoding that is used to encode the HL7 data") + .required(true) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .build(); + + static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("Any FlowFile that cannot be parsed as HL7 will be routed to this relationship") + .build(); + static final Relationship REL_ORIGINAL = new Relationship.Builder() + .name("original") + .description("The original FlowFile that comes into this processor will be routed to this relationship, unless it is routed to 'failure'") + .build(); + + private volatile Map queries = new HashMap<>(); + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .description("Specifies a query that will cause any HL7 message matching the query to be routed to the '" + propertyDescriptorName + "' relationship") + .required(false) + .dynamic(true) + .addValidator(new HL7QueryValidator()) + .build(); + } + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(); + properties.add(CHARACTER_SET); + return properties; + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + if ( !descriptor.isDynamic() ) { + return; + } + + final Map updatedQueryMap = new HashMap<>(queries); + final Relationship relationship = new Relationship.Builder().name(descriptor.getName()).build(); + + if ( newValue == null ) { + updatedQueryMap.remove(relationship); + } else { + final HL7Query query = HL7Query.compile(newValue); + updatedQueryMap.put(relationship, query); + } + + this.queries = updatedQueryMap; + } + + @Override + public Set getRelationships() { + final Set relationships = new HashSet<>(queries.keySet()); + relationships.add(REL_FAILURE); + relationships.add(REL_ORIGINAL); + return relationships; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).evaluateAttributeExpressions(flowFile).getValue()); + + final byte[] buffer = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, buffer); + } + }); + + @SuppressWarnings("resource") + final HapiContext hapiContext = new DefaultHapiContext(); + hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + + final PipeParser parser = hapiContext.getPipeParser(); + final String hl7Text = new String(buffer, charset); + final HL7Message message; + try { + final Message hapiMessage = parser.parse(hl7Text); + message = new HapiMessage(hapiMessage); + } catch (final Exception e) { + getLogger().error("Failed to parse {} as HL7 due to {}; routing to failure", new Object[] {flowFile, e}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + final Set matchingRels = new HashSet<>(); + final Map queryMap = queries; + for ( final Map.Entry entry : queryMap.entrySet() ) { + final Relationship relationship = entry.getKey(); + final HL7Query query = entry.getValue(); + + final QueryResult result = query.evaluate(message); + if ( result.isMatch() ) { + FlowFile clone = session.clone(flowFile); + clone = session.putAttribute(clone, "RouteHL7.Route", relationship.getName()); + session.transfer(clone, relationship); + session.getProvenanceReporter().route(clone, relationship); + matchingRels.add(relationship.getName()); + } + } + + session.transfer(flowFile, REL_ORIGINAL); + getLogger().info("Routed a copy of {} to {} relationships: {}", new Object[] {flowFile, matchingRels.size(), matchingRels}); + } + + private static class HL7QueryValidator implements Validator { + + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + String error = null; + + try { + final HL7Query hl7Query = HL7Query.compile(input); + final List> returnTypes = hl7Query.getReturnTypes(); + if ( returnTypes.size() != 1 ) { + error = "RouteHL7 requires that the HL7 Query return exactly 1 element of type MESSAGE"; + } else if ( !HL7Message.class.isAssignableFrom(returnTypes.get(0)) ) { + error = "RouteHL7 requires that the HL7 Query return exactly 1 element of type MESSAGE"; + } + } catch (final HL7QueryParsingException e) { + error = e.toString(); + } + + if ( error == null ) { + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } else { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(error).build(); + } + } + + } +} diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..3f57ff02f1 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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.processors.hl7.ExtractHL7Attributes +org.apache.nifi.processors.hl7.RouteHL7 \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java new file mode 100644 index 0000000000..f56628811e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java @@ -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.processors.hl7; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.nifi.processors.hl7.ExtractHL7Attributes; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; + +public class TestExtractHL7Attributes { + + @Test + public void testExtract() throws IOException { + System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", "DEBUG"); + final TestRunner runner = TestRunners.newTestRunner(ExtractHL7Attributes.class); + runner.enqueue(Paths.get("src/test/resources/1.hl7")); + + runner.run(); + runner.assertAllFlowFilesTransferred(ExtractHL7Attributes.REL_SUCCESS, 1); + final MockFlowFile out = runner.getFlowFilesForRelationship(ExtractHL7Attributes.REL_SUCCESS).get(0); + final SortedMap sortedAttrs = new TreeMap<>(out.getAttributes()); + for (final Map.Entry entry : sortedAttrs.entrySet()) { + System.out.println(entry.getKey() + " : " + entry.getValue()); + } + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 new file mode 100644 index 0000000000..bf2b8a581d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 @@ -0,0 +1,16 @@ +MSH|^~`&|ECG REPORTING|ROCHESTER|ERIS|ROCHESTER|20110621050440||ORU^R01|20110621050440|P|2.1 +PID|||999999999||TEST^PATIENT||18450101|F +OBR|||211088491|0^ADULT^ROCHECG|||20110620170631|||||||||M999999^^^^^^^RACFID||||||20110621060232||EC|F|||||||M999999^LASTNAME MD^FIRSTNAME^^^^^RACFID +OBX||ST|93000.2^VENTRICULAR RATE EKG/MIN^CPT4|1|52|/SEC +OBX||ST|93000.4^PR INTERVAL(MSEC)^CPT4|2|208|MSEC +OBX||ST|93000.5^QRS - INTERVAL(MSEC)^CPT4|3|88|MSEC +OBX||ST|93000.6^QT - INTERVAL(MSEC)^CPT4|4|466|MSEC +OBX||ST|93000&PTL^PHYSICAL TEST LOCATION^CPT4|5|STMA +OBX||ST|93000&PTR^PHYSICAL TEST ROOM^CPT4|6|04254 +OBX||CE|93000.17^^CPT4|7|21&101^Sinus bradycardia`T`with 1st degree A-V block^MEIECG +OBX||CE|93000.17^^CPT4|8|1687^Otherwise normal ECG^MEIECG +OBX||CE|93000&CMP^^CPT4|9|1301^When compared with ECG of^MEIECG +OBX||TS|93000&CMD^EKG COMPARISON DATE^CPT4|10|201106171659 +OBX||CE|93000&CMP^^CPT4|11|1305^No significant change was found^MEIECG +OBX||TX|93000.48^EKG COMMENT^CPT4|12|9917^LASTNAME MD^FIRSTNAME +OBX||FT|93000^ECG 12-LEAD^CPT4|13|{\rtf1\ansi \deff1\deflang1033\ {\fonttbl{\f1\fmodern\fcharset0 Courier;}{\f2\fmodern\fcharset0 Courier;}} \pard\plain \f1\fs18\par 20Jun2011 17:06\par VENTRICULAR RATE 52\par Sinus bradycardia with 1st degree A-V block\par Otherwise normal ECG\par When compared with ECG of 17-JUN-2011 16:59,\par No significant change was found\par 47507`S`'LASTNAME MD`S`'FIRSTNAME \par } \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 new file mode 100644 index 0000000000..02e8967296 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 @@ -0,0 +1,5 @@ +MSH|^~\&|CERNER||PriorityHealth||||ORU^R01|Q479004375T431430612|P|2.3| +PID|||001677980||SMITH^CURTIS||19680219|M||||||||||929645156318|123456789| +PD1||||1234567890^LAST^FIRST^M^^^^^NPI| +OBR|1|341856649^HNAM_ORDERID|000002006326002362|648088^Basic Metabolic Panel|||20061122151600|||||||||1620^Hooker^Robert^L||||||20061122154733|||F|||||||||||20061122140000| +OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20061122154733| \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-hl7-bundle/pom.xml new file mode 100644 index 0000000000..3f9fbce211 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 0.1.0-incubating-SNAPSHOT + + + nifi-hl7-bundle + pom + + + nifi-hl7-processors + nifi-hl7-nar + + + From ff0bd2c6696e73d3157eb2d0116913e337410f38 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 9 Apr 2015 17:54:59 -0400 Subject: [PATCH 06/16] 504: Initial import of geo enrich processors --- .../nifi-geo-bundle/nifi-geo-nar/pom.xml | 33 ++ .../nifi-geo-processors/.gitignore | 1 + .../nifi-geo-processors/pom.xml | 43 +++ .../apache/nifi/processors/GeoEnrichIP.java | 210 +++++++++++++ .../processors/maxmind/DatabaseReader.java | 286 ++++++++++++++++++ .../org.apache.nifi.processor.Processor | 16 + nifi/nifi-nar-bundles/nifi-geo-bundle/pom.xml | 42 +++ 7 files changed, 631 insertions(+) create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-nar/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/.gitignore create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/GeoEnrichIP.java create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi/nifi-nar-bundles/nifi-geo-bundle/pom.xml diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-nar/pom.xml new file mode 100644 index 0000000000..484e291369 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-nar/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-geo-bundle + 0.1.0-incubating-SNAPSHOT + + nifi-geo-nar + nar + NiFi Geo Enrichment NAR + + + + org.apache.nifi + nifi-geo-processors + + + diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/.gitignore b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/.gitignore new file mode 100644 index 0000000000..b83d22266a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/pom.xml new file mode 100644 index 0000000000..67bc253a67 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/pom.xml @@ -0,0 +1,43 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-geo-bundle + 0.1.0-incubating-SNAPSHOT + + nifi-geo-processors + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-utils + + + com.maxmind.geoip2 + geoip2 + 2.1.0 + + + diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/GeoEnrichIP.java b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/GeoEnrichIP.java new file mode 100644 index 0000000000..fed0e7e601 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/GeoEnrichIP.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +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.flowfile.FlowFile; +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.util.StandardValidators; +import org.apache.nifi.processors.maxmind.DatabaseReader; +import org.apache.nifi.util.StopWatch; + +import com.maxmind.geoip2.exception.GeoIp2Exception; +import com.maxmind.geoip2.model.CityResponse; +import com.maxmind.geoip2.record.Subdivision; + +@EventDriven +@SideEffectFree +@SupportsBatching +@Tags({"geo", "enrich", "ip", "maxmind"}) +@CapabilityDescription("Looks up geolocation information for an IP address and adds the geo information to FlowFile attributes. The " + + "geo data is provided as a MaxMind database. The attribute that contains the IP address to lookup is provided by the " + + "'IP Address Attribute' property. If the name of the attribute provided is 'X', then the the attributes added by enrichment " + + "will take the form X.geo.") +@WritesAttributes({ + @WritesAttribute(attribute="X.geo.lookup.micros", description="The number of microseconds that the geo lookup took"), + @WritesAttribute(attribute="X.geo.city", description="The city identified for the IP address"), + @WritesAttribute(attribute="X.geo.latitude", description="The latitude identified for this IP address"), + @WritesAttribute(attribute="X.geo.longitude", description="The longitude identified for this IP address"), + @WritesAttribute(attribute="X.geo.subdivision.N", description="Each subdivision that is identified for this IP address is added with a one-up number appended to the attribute name, starting with 0"), + @WritesAttribute(attribute="X.geo.subdivision.isocode.N", description="The ISO code for the subdivision that is identified by X.geo.subdivision.N"), + @WritesAttribute(attribute="X.geo.country", description="The country identified for this IP address"), + @WritesAttribute(attribute="X.geo.country.isocode", description="The ISO Code for the country identified"), + @WritesAttribute(attribute="X.geo.postalcode", description="The postal code for the country identified"), +}) +public class GeoEnrichIP extends AbstractProcessor { + + public static final PropertyDescriptor GEO_DATABASE_FILE = new PropertyDescriptor.Builder() + .name("Geo Database File") + .description("Path to Maxmind Geo Enrichment Database File") + .required(true) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .build(); + + public static final PropertyDescriptor IP_ADDRESS_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("IP Address Attribute") + .required(true) + .description("The name of an attribute whose value is a dotted decimal IP address for which enrichment should occur") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final Relationship REL_FOUND = new Relationship.Builder() + .name("found") + .description("Where to route flow files after successfully enriching attributes with geo data") + .build(); + + public static final Relationship REL_NOT_FOUND = new Relationship.Builder() + .name("not found") + .description("Where to route flow files after unsuccessfully enriching attributes because no geo data was found") + .build(); + + private Set relationships; + private List propertyDescriptors; + private final AtomicReference databaseReaderRef = new AtomicReference<>(null); + + @Override + public Set getRelationships() { + return relationships; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return propertyDescriptors; + } + + @OnScheduled + public final void onScheduled(final ProcessContext context) throws IOException { + final String dbFileString = context.getProperty(GEO_DATABASE_FILE).getValue(); + final File dbFile = new File(dbFileString); + final StopWatch stopWatch = new StopWatch(true); + final DatabaseReader reader = new DatabaseReader.Builder(dbFile).build(); + stopWatch.stop(); + getLogger().info("Completed loading of Maxmind Geo Database. Elapsed time was {} milliseconds.", new Object[]{stopWatch.getDuration(TimeUnit.MILLISECONDS)}); + databaseReaderRef.set(reader); + } + + @OnStopped + public void closeReader() throws IOException { + final DatabaseReader reader = databaseReaderRef.get(); + if ( reader != null ) { + reader.close(); + } + } + + @Override + protected void init(final ProcessorInitializationContext context) { + final Set rels = new HashSet<>(); + rels.add(REL_FOUND); + rels.add(REL_NOT_FOUND); + this.relationships = Collections.unmodifiableSet(rels); + + final List props = new ArrayList<>(); + props.add(GEO_DATABASE_FILE); + props.add(IP_ADDRESS_ATTRIBUTE); + this.propertyDescriptors = Collections.unmodifiableList(props); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + final DatabaseReader dbReader = databaseReaderRef.get(); + final String ipAttributeName = context.getProperty(IP_ADDRESS_ATTRIBUTE).getValue(); + final String ipAttributeValue = flowFile.getAttribute(ipAttributeName); + if (StringUtils.isEmpty(ipAttributeName)) { //TODO need to add additional validation - should look like an IPv4 or IPv6 addr for instance + session.transfer(flowFile, REL_NOT_FOUND); + getLogger().warn("Unable to find ip address for {}", new Object[]{flowFile}); + return; + } + InetAddress inetAddress = null; + CityResponse response = null; + + try { + inetAddress = InetAddress.getByName(ipAttributeValue); + } catch (final IOException ioe) { + session.transfer(flowFile, REL_NOT_FOUND); + getLogger().warn("Could not resolve {} to ip address for {}", new Object[]{ipAttributeValue, flowFile}, ioe); + return; + } + final StopWatch stopWatch = new StopWatch(true); + try { + response = dbReader.city(inetAddress); + stopWatch.stop(); + } catch (final IOException | GeoIp2Exception ex) { + session.transfer(flowFile, REL_NOT_FOUND); + getLogger().warn("Failure while trying to find enrichment data for {} due to {}", new Object[]{flowFile, ex}, ex); + return; + } + + if (response == null) { + session.transfer(flowFile, REL_NOT_FOUND); + getLogger().warn("No enrichment data found for ip {} of {}", new Object[]{ipAttributeValue, flowFile}); + return; + } + + final Map attrs = new HashMap<>(); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.lookup.micros").toString(), String.valueOf(stopWatch.getDuration(TimeUnit.MICROSECONDS))); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.city").toString(), response.getCity().getName()); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.latitude").toString(), response.getLocation().getLatitude().toString()); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.longitude").toString(), response.getLocation().getLongitude().toString()); + int i = 0; + for (final Subdivision subd : response.getSubdivisions()) { + attrs.put(new StringBuilder(ipAttributeName).append(".geo.subdivision.").append(i).toString(), subd.getName()); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.subdivision.isocode.").append(i).toString(), subd.getIsoCode()); + i++; + } + attrs.put(new StringBuilder(ipAttributeName).append(".geo.country").toString(), response.getCountry().getName()); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.country.isocode").toString(), response.getCountry().getIsoCode()); + attrs.put(new StringBuilder(ipAttributeName).append(".geo.postalcode").toString(), response.getPostal().getCode()); + flowFile = session.putAllAttributes(flowFile, attrs); + + session.transfer(flowFile, REL_FOUND); + getLogger().info("Completed lookup of IP geo information for {}", new Object[]{flowFile}); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java new file mode 100644 index 0000000000..796a7aff5c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java @@ -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.processors.maxmind; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.InetAddress; +import java.util.Arrays; +import java.util.List; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.maxmind.db.Metadata; +import com.maxmind.db.Reader; +import com.maxmind.db.Reader.FileMode; +import com.maxmind.geoip2.GeoIp2Provider; +import com.maxmind.geoip2.exception.AddressNotFoundException; +import com.maxmind.geoip2.exception.GeoIp2Exception; +import com.maxmind.geoip2.model.AnonymousIpResponse; +import com.maxmind.geoip2.model.CityResponse; +import com.maxmind.geoip2.model.ConnectionTypeResponse; +import com.maxmind.geoip2.model.CountryResponse; +import com.maxmind.geoip2.model.DomainResponse; +import com.maxmind.geoip2.model.IspResponse; + +/** + *

+ * This class was copied from + * https://raw.githubusercontent.com/maxmind/GeoIP2-java/master/src/main/java/com/maxmind/geoip2/DatabaseReader.java + * It is written by Maxmind and it is available under Apache Software License V2 + * Copyright (c) 2013 by MaxMind, Inc. + * The modification we're making to the code below is to stop using exceptions for + * mainline flow control. Specifically we don't want to throw an exception + * simply because an address was not found. + *

+ * + * Instances of this class provide a reader for the GeoIP2 database format. IP + * addresses can be looked up using the get method. + */ +public class DatabaseReader implements GeoIp2Provider, Closeable { + + private final Reader reader; + + private final ObjectMapper om; + + private DatabaseReader(Builder builder) throws IOException { + if (builder.stream != null) { + this.reader = new Reader(builder.stream); + } else if (builder.database != null) { + this.reader = new Reader(builder.database, builder.mode); + } else { + // This should never happen. If it does, review the Builder class + // constructors for errors. + throw new IllegalArgumentException( + "Unsupported Builder configuration: expected either File or URL"); + } + this.om = new ObjectMapper(); + this.om.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, + false); + this.om.configure( + DeserializationFeature.READ_UNKNOWN_ENUM_VALUES_AS_NULL, true); + InjectableValues inject = new InjectableValues.Std().addValue( + "locales", builder.locales); + this.om.setInjectableValues(inject); + } + + /** + *

+ * Constructs a Builder for the DatabaseReader. The file passed to it must + * be a valid GeoIP2 database file. + *

+ *

+ * Builder creates instances of DatabaseReader + * from values set by the methods. + *

+ *

+ * Only the values set in the Builder constructor are required. + *

+ */ + public final static class Builder { + final File database; + final InputStream stream; + + List locales = Arrays.asList("en"); + FileMode mode = FileMode.MEMORY_MAPPED; + + /** + * @param stream the stream containing the GeoIP2 database to use. + */ + public Builder(InputStream stream) { + this.stream = stream; + this.database = null; + } + + /** + * @param database the GeoIP2 database file to use. + */ + public Builder(File database) { + this.database = database; + this.stream = null; + } + + /** + * @param val List of locale codes to use in name property from most + * preferred to least preferred. + * @return Builder object + */ + public Builder locales(List val) { + this.locales = val; + return this; + } + + /** + * @param val The file mode used to open the GeoIP2 database + * @return Builder object + * @throws java.lang.IllegalArgumentException if you initialized the Builder with a URL, which uses + * {@link FileMode#MEMORY}, but you provided a different + * FileMode to this method. + */ + public Builder fileMode(FileMode val) { + if (this.stream != null && !FileMode.MEMORY.equals(val)) { + throw new IllegalArgumentException( + "Only FileMode.MEMORY is supported when using an InputStream."); + } + this.mode = val; + return this; + } + + /** + * @return an instance of DatabaseReader created from the + * fields set on this builder. + * @throws IOException if there is an error reading the database + */ + public DatabaseReader build() throws IOException { + return new DatabaseReader(this); + } + } + + /** + * @param ipAddress IPv4 or IPv6 address to lookup. + * @return A object with the data for the IP address or null if no + * information could be found for the given IP address + * @throws IOException if there is an error opening or reading from the file. + */ + private T get(InetAddress ipAddress, Class cls, boolean hasTraits, + String type) throws IOException, AddressNotFoundException { + + String databaseType = this.getMetadata().getDatabaseType(); + if (!databaseType.contains(type)) { + String caller = Thread.currentThread().getStackTrace()[2] + .getMethodName(); + throw new UnsupportedOperationException( + "Invalid attempt to open a " + databaseType + + " database using the " + caller + " method"); + } + + ObjectNode node = (ObjectNode) this.reader.get(ipAddress); + + if (node == null) { + return null; + } + + ObjectNode ipNode; + if (hasTraits) { + if (!node.has("traits")) { + node.set("traits", this.om.createObjectNode()); + } + ipNode = (ObjectNode) node.get("traits"); + } else { + ipNode = node; + } + ipNode.put("ip_address", ipAddress.getHostAddress()); + + return this.om.treeToValue(node, cls); + } + + /** + *

+ * Closes the database. + *

+ *

+ * If you are using FileMode.MEMORY_MAPPED, this will + * not unmap the underlying file due to a limitation in Java's + * MappedByteBuffer. It will however set the reference to + * the buffer to null, allowing the garbage collector to + * collect it. + *

+ * + * @throws IOException if an I/O error occurs. + */ + @Override + public void close() throws IOException { + this.reader.close(); + } + + @Override + public CountryResponse country(InetAddress ipAddress) throws IOException, + GeoIp2Exception { + return this.get(ipAddress, CountryResponse.class, true, "Country"); + } + + @Override + public CityResponse city(InetAddress ipAddress) throws IOException, + GeoIp2Exception { + return this.get(ipAddress, CityResponse.class, true, "City"); + } + + /** + * Look up an IP address in a GeoIP2 Anonymous IP. + * + * @param ipAddress IPv4 or IPv6 address to lookup. + * @return a AnonymousIpResponse for the requested IP address. + * @throws GeoIp2Exception if there is an error looking up the IP + * @throws IOException if there is an IO error + */ + public AnonymousIpResponse anonymousIp(InetAddress ipAddress) throws IOException, + GeoIp2Exception { + return this.get(ipAddress, AnonymousIpResponse.class, false, "GeoIP2-Anonymous-IP"); + } + + /** + * Look up an IP address in a GeoIP2 Connection Type database. + * + * @param ipAddress IPv4 or IPv6 address to lookup. + * @return a ConnectTypeResponse for the requested IP address. + * @throws GeoIp2Exception if there is an error looking up the IP + * @throws IOException if there is an IO error + */ + public ConnectionTypeResponse connectionType(InetAddress ipAddress) + throws IOException, GeoIp2Exception { + return this.get(ipAddress, ConnectionTypeResponse.class, false, + "GeoIP2-Connection-Type"); + } + + /** + * Look up an IP address in a GeoIP2 Domain database. + * + * @param ipAddress IPv4 or IPv6 address to lookup. + * @return a DomainResponse for the requested IP address. + * @throws GeoIp2Exception if there is an error looking up the IP + * @throws IOException if there is an IO error + */ + public DomainResponse domain(InetAddress ipAddress) throws IOException, + GeoIp2Exception { + return this + .get(ipAddress, DomainResponse.class, false, "GeoIP2-Domain"); + } + + /** + * Look up an IP address in a GeoIP2 ISP database. + * + * @param ipAddress IPv4 or IPv6 address to lookup. + * @return an IspResponse for the requested IP address. + * @throws GeoIp2Exception if there is an error looking up the IP + * @throws IOException if there is an IO error + */ + public IspResponse isp(InetAddress ipAddress) throws IOException, + GeoIp2Exception { + return this.get(ipAddress, IspResponse.class, false, "GeoIP2-ISP"); + } + + /** + * @return the metadata for the open MaxMind DB file. + */ + public Metadata getMetadata() { + return this.reader.getMetadata(); + } +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..9b1be71888 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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.processors.GeoEnrichIP \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-geo-bundle/pom.xml new file mode 100644 index 0000000000..2dbd32fe91 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/pom.xml @@ -0,0 +1,42 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-nar-bundles + 0.1.0-incubating-SNAPSHOT + + + nifi-geo-bundle + pom + NiFi Geo Enrichment Capability Set + + + nifi-geo-processors + nifi-geo-nar + + + + + + org.apache.nifi + nifi-geo-processors + 0.1.0-incubating-SNAPSHOT + + + + From 178c5cd287eed734bd3d93665df27682db941a8b Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 9 Apr 2015 17:55:33 -0400 Subject: [PATCH 07/16] NIFI-505: Initial import of language translation nar --- .../nifi-language-translation-nar/pom.xml | 36 ++ .../nifi-yandex-processors/.gitignore | 1 + .../nifi-yandex-processors/pom.xml | 63 ++++ .../processors/yandex/YandexTranslate.java | 325 ++++++++++++++++++ .../processors/yandex/model/Translation.java | 52 +++ .../processors/yandex/util/Languages.java | 86 +++++ .../yandex/util/ObjectMapperResolver.java | 48 +++ .../org.apache.nifi.processor.Processor | 16 + .../yandex/TestYandexTranslate.java | 141 ++++++++ .../nifi-language-translation-bundle/pom.xml | 48 +++ 10 files changed, 816 insertions(+) create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/.gitignore create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/model/Translation.java create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/Languages.java create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/ObjectMapperResolver.java create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java create mode 100644 nifi/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml new file mode 100644 index 0000000000..4d8b790a9c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-language-translation-nar/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-language-translation-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-language-translation-nar + nar + + + + org.apache.nifi + nifi-yandex-processors + 0.1.0-incubating-SNAPSHOT + + + + diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/.gitignore b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/.gitignore new file mode 100644 index 0000000000..b83d22266a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml new file mode 100644 index 0000000000..a5f9f0e3a2 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/pom.xml @@ -0,0 +1,63 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-language-translation-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-yandex-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + + com.sun.jersey + jersey-client + + + com.sun.jersey + jersey-json + + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java new file mode 100644 index 0000000000..a5eecc6238 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java @@ -0,0 +1,325 @@ +/* + * 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.yandex; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.Charset; +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.Set; + +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; + +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +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.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.InputStreamCallback; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.yandex.model.Translation; +import org.apache.nifi.processors.yandex.util.Languages; +import org.apache.nifi.processors.yandex.util.ObjectMapperResolver; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.StopWatch; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.ClientResponse.Status; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; +import com.sun.jersey.api.json.JSONConfiguration; +import com.sun.jersey.core.util.MultivaluedMapImpl; + +@SupportsBatching +@Tags({"yandex", "translate", "translation", "language"}) +@CapabilityDescription("Translates content and attributes from one language to another") +@WritesAttributes({ + @WritesAttribute(attribute="yandex.translate.failure.reason", description="If the text cannot be translated, this attribute will be set indicating the reason for the failure"), + @WritesAttribute(attribute="language", description="When the translation succeeds, if the content was translated, this attribute will be set indicating the new language of the content") +}) +@DynamicProperty(name="The name of an attribute to set that will contain the translated text of the value", + value="The value to translate", + supportsExpressionLanguage=true, + description="User-defined properties are used to translate arbitrary text based on attributes.") +public class YandexTranslate extends AbstractProcessor { + + public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder() + .name("Yandex API Key") + .description("The API Key that is registered with Yandex") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .required(true) + .build(); + public static final PropertyDescriptor SOURCE_LANGUAGE = new PropertyDescriptor.Builder() + .name("Input Language") + .description("The language of incoming data") + .required(true) + .defaultValue("sp") + .expressionLanguageSupported(true) + .addValidator(new LanguageNameValidator()) + .build(); + public static final PropertyDescriptor TARGET_LANGUAGE = new PropertyDescriptor.Builder() + .name("Target Language") + .description("The language to translate the text into") + .required(true) + .defaultValue("en") + .expressionLanguageSupported(true) + .addValidator(new LanguageNameValidator()) + .build(); + public static final PropertyDescriptor TRANSLATE_CONTENT = new PropertyDescriptor.Builder() + .name("Translate Content") + .description("Specifies whether or not the content should be translated. If false, only the text specified by user-defined properties will be translated.") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() + .name("Character Set") + .description("Specifies the character set of the data to be translated") + .required(true) + .defaultValue("UTF-8") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("This relationship is used when the translation is successful") + .build(); + public static final Relationship REL_COMMS_FAILURE = new Relationship.Builder() + .name("comms.failure") + .description("This relationship is used when the translation fails due to a problem such as a network failure, and for which the translation should be attempted again") + .build(); + public static final Relationship REL_TRANSLATION_FAILED = new Relationship.Builder() + .name("translation.failure") + .description("This relationship is used if the translation cannot be performed for some reason other than communications failure") + .build(); + + private List descriptors; + private Set relationships; + + private volatile Client client; + + private static final String URL = "https://translate.yandex.net/api/v1.5/tr.json/translate"; + + @Override + protected void init(final ProcessorInitializationContext context) { + final List descriptors = new ArrayList(); + descriptors.add(KEY); + descriptors.add(SOURCE_LANGUAGE); + descriptors.add(TARGET_LANGUAGE); + descriptors.add(TRANSLATE_CONTENT); + descriptors.add(CHARACTER_SET); + this.descriptors = Collections.unmodifiableList(descriptors); + + final Set relationships = new HashSet(); + relationships.add(REL_SUCCESS); + relationships.add(REL_COMMS_FAILURE); + relationships.add(REL_TRANSLATION_FAILED); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return this.relationships; + } + + @Override + public final List getSupportedPropertyDescriptors() { + return descriptors; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .dynamic(true) + .build(); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final List results = new ArrayList<>(); + if ( validationContext.getProperty(TRANSLATE_CONTENT).asBoolean().equals(Boolean.FALSE) ) { + boolean foundDynamic = false; + for ( final PropertyDescriptor descriptor : validationContext.getProperties().keySet() ) { + if ( descriptor.isDynamic() ) { + foundDynamic = true; + break; + } + } + + if ( !foundDynamic ) { + results.add(new ValidationResult.Builder().subject("Text to translate").input("").valid(false).explanation("Must either set 'Translate Content' to true or add at least one user-defined property").build()); + } + } + + return results; + } + + @OnScheduled + public void onScheduled(final ProcessContext context) { + final ClientConfig config = new DefaultClientConfig(); + config.getFeatures().put(JSONConfiguration.FEATURE_POJO_MAPPING, Boolean.TRUE); + config.getClasses().add(ObjectMapperResolver.class); + + client = Client.create(config); + } + + @OnStopped + public void destroyClient() { + if ( client != null ) { + client.destroy(); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + final StopWatch stopWatch = new StopWatch(true); + final String key = context.getProperty(KEY).getValue(); + final String sourceLanguage = context.getProperty(SOURCE_LANGUAGE).evaluateAttributeExpressions(flowFile).getValue(); + final String targetLanguage = context.getProperty(TARGET_LANGUAGE).evaluateAttributeExpressions(flowFile).getValue(); + final String encoding = context.getProperty(CHARACTER_SET).evaluateAttributeExpressions(flowFile).getValue(); + + final List attributeNames = new ArrayList<>(); + final List textValues = new ArrayList<>(); + for ( final PropertyDescriptor descriptor : context.getProperties().keySet() ) { + if ( descriptor.isDynamic() ) { + attributeNames.add(descriptor.getName()); // add to list so that we know the order when the translations come back. + textValues.add(context.getProperty(descriptor).evaluateAttributeExpressions(flowFile).getValue()); + } + } + + if ( context.getProperty(TRANSLATE_CONTENT).asBoolean() ) { + final byte[] buff = new byte[(int) flowFile.getSize()]; + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream in) throws IOException { + StreamUtils.fillBuffer(in, buff); + } + }); + final String content = new String(buff, Charset.forName(encoding)); + textValues.add(content); + } + + WebResource webResource = client.resource(URL); + + final MultivaluedMap paramMap = new MultivaluedMapImpl(); + paramMap.put("text", textValues); + paramMap.add("key", key); + paramMap.add("lang", sourceLanguage + "-" + targetLanguage); + + WebResource.Builder builder = webResource + .accept(MediaType.APPLICATION_JSON) + .type(MediaType.APPLICATION_FORM_URLENCODED); + builder = builder.entity(paramMap); + + final ClientResponse response; + try { + response = builder.post(ClientResponse.class); + } catch (final Exception e) { + getLogger().error("Failed to make request to Yandex to transate text for {} due to {}; routing to comms.failure", new Object[] {flowFile, e}); + session.transfer(flowFile, REL_COMMS_FAILURE); + return; + } + + if ( response.getStatus() != Status.OK.getStatusCode() ) { + getLogger().error("Failed to translate text using Yandex for {}; response was {}: {}; routing to {}", new Object[] { + flowFile, response.getStatus(), response.getStatusInfo().getReasonPhrase(), REL_TRANSLATION_FAILED.getName()}); + flowFile = session.putAttribute(flowFile, "yandex.translate.failure.reason", response.getStatusInfo().getReasonPhrase()); + session.transfer(flowFile, REL_TRANSLATION_FAILED); + return; + } + + final Map newAttributes = new HashMap<>(); + final Translation translation = response.getEntity(Translation.class); + final List texts = translation.getText(); + for (int i=0; i < texts.size(); i++) { + final String text = texts.get(i); + if ( i < attributeNames.size() ) { + final String attributeName = attributeNames.get(i); + newAttributes.put(attributeName, text); + } else { + flowFile = session.write(flowFile, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + out.write(text.getBytes(encoding)); + } + }); + + newAttributes.put("language", targetLanguage); + } + } + + if ( !newAttributes.isEmpty() ) { + flowFile = session.putAllAttributes(flowFile, newAttributes); + } + + stopWatch.stop(); + session.transfer(flowFile, REL_SUCCESS); + getLogger().info("Successfully translated {} items for {} from {} to {} in {}; routing to success", new Object[] {texts.size(), flowFile, sourceLanguage, targetLanguage, stopWatch.getDuration()}); + } + + + private static class LanguageNameValidator implements Validator { + + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).valid(true).explanation("Expression Language Present").build(); + } + + if ( Languages.getLanguageMap().keySet().contains(input.toLowerCase()) ) { + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } + + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(input + " is not a language that is supported by Yandex").build(); + } + + } +} diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/model/Translation.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/model/Translation.java new file mode 100644 index 0000000000..eeb7c3fa52 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/model/Translation.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.yandex.model; + +import java.util.List; + +import javax.xml.bind.annotation.XmlRootElement; + +@XmlRootElement(name = "translation") +public class Translation { + private int code; + private String lang; + private List text; + + public int getCode() { + return code; + } + + public void setCode(final int code) { + this.code = code; + } + + public String getLang() { + return lang; + } + + public void setLang(final String lang) { + this.lang = lang; + } + + public List getText() { + return text; + } + + public void setText(final List text) { + this.text = text; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/Languages.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/Languages.java new file mode 100644 index 0000000000..791d6a3910 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/Languages.java @@ -0,0 +1,86 @@ +/* + * 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.yandex.util; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class Languages { + private static final Map languageAbbreviationMap = new HashMap<>(); + + static { + languageAbbreviationMap.put("ar", "arabic"); + languageAbbreviationMap.put("az", "azerbaijani"); + languageAbbreviationMap.put("be", "belarusian"); + languageAbbreviationMap.put("bg", "bulgarian"); + languageAbbreviationMap.put("bs", "bosnian"); + languageAbbreviationMap.put("ca", "catalan"); + languageAbbreviationMap.put("cs", "czech"); + languageAbbreviationMap.put("da", "danish"); + languageAbbreviationMap.put("de", "german"); + languageAbbreviationMap.put("el", "greek"); + languageAbbreviationMap.put("en", "english"); + languageAbbreviationMap.put("es", "spanish"); + languageAbbreviationMap.put("et", "estonian"); + languageAbbreviationMap.put("fi", "finnish"); + languageAbbreviationMap.put("fr", "french"); + languageAbbreviationMap.put("he", "hebrew"); + languageAbbreviationMap.put("hr", "croatian"); + languageAbbreviationMap.put("hu", "hungarian"); + languageAbbreviationMap.put("hy", "armenian"); + languageAbbreviationMap.put("id", "indonesian"); + languageAbbreviationMap.put("is", "icelandic"); + languageAbbreviationMap.put("it", "italian"); + languageAbbreviationMap.put("ja", "japanese"); + languageAbbreviationMap.put("ka", "georgian"); + languageAbbreviationMap.put("ko", "korean"); + languageAbbreviationMap.put("lt", "lithuanian"); + languageAbbreviationMap.put("lv", "latvian"); + languageAbbreviationMap.put("mk", "macedonian"); + languageAbbreviationMap.put("ms", "malay"); + languageAbbreviationMap.put("mt", "maltese"); + languageAbbreviationMap.put("nl", "dutch"); + languageAbbreviationMap.put("no", "norwegian"); + languageAbbreviationMap.put("pl", "polish"); + languageAbbreviationMap.put("pt", "portuguese"); + languageAbbreviationMap.put("ro", "romanian"); + languageAbbreviationMap.put("ru", "russian"); + languageAbbreviationMap.put("sk", "slovak"); + languageAbbreviationMap.put("sl", "slovenian"); + languageAbbreviationMap.put("sq", "albanian"); + languageAbbreviationMap.put("sr", "serbian"); + languageAbbreviationMap.put("sv", "swedish"); + languageAbbreviationMap.put("th", "thai"); + languageAbbreviationMap.put("tr", "turkish"); + languageAbbreviationMap.put("uk", "ukrainian"); + languageAbbreviationMap.put("vi", "vietnamese"); + languageAbbreviationMap.put("zh", "chinese"); + + final Map reverseMap = new HashMap<>(); + for ( final Map.Entry entry : languageAbbreviationMap.entrySet() ) { + reverseMap.put(entry.getValue(), entry.getKey()); + } + + languageAbbreviationMap.putAll(reverseMap); + } + + + public static Map getLanguageMap() { + return Collections.unmodifiableMap(languageAbbreviationMap); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/ObjectMapperResolver.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/ObjectMapperResolver.java new file mode 100644 index 0000000000..085cf7f080 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/util/ObjectMapperResolver.java @@ -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.processors.yandex.util; + +import javax.ws.rs.ext.ContextResolver; +import javax.ws.rs.ext.Provider; +import org.codehaus.jackson.map.AnnotationIntrospector; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.SerializationConfig; +import org.codehaus.jackson.map.annotate.JsonSerialize.Inclusion; +import org.codehaus.jackson.xc.JaxbAnnotationIntrospector; + +@Provider +public class ObjectMapperResolver implements ContextResolver { + + private final ObjectMapper mapper; + + public ObjectMapperResolver() throws Exception { + mapper = new ObjectMapper(); + + final AnnotationIntrospector jaxbIntrospector = new JaxbAnnotationIntrospector(); + final SerializationConfig serializationConfig = mapper.getSerializationConfig(); + final DeserializationConfig deserializationConfig = mapper.getDeserializationConfig(); + + mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector)); + mapper.setDeserializationConfig(deserializationConfig.withAnnotationIntrospector(jaxbIntrospector)); + } + + @Override + public ObjectMapper getContext(Class objectType) { + return mapper; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..dc81439452 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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.processors.yandex.YandexTranslate \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java new file mode 100644 index 0000000000..4f82049353 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java @@ -0,0 +1,141 @@ +/* + * 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.yandex; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Properties; + +import org.apache.nifi.processors.yandex.YandexTranslate; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +@Ignore("For local testing only; requires local file to be populated with Yandex API Key") +public class TestYandexTranslate { + + private TestRunner testRunner; + private String apiKey; + + @Before + public void init() throws IOException { + testRunner = TestRunners.newTestRunner(YandexTranslate.class); + + final Properties properties = new Properties(); + try (final InputStream in = new FileInputStream(new File("C:/dev/notes/yandex-info.txt"))) { + properties.load(in); + } + apiKey = properties.getProperty("api_key").trim(); + } + + + @Test + public void testTranslateContent() { + testRunner.setProperty(YandexTranslate.KEY, apiKey); + testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); + testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); + testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "true"); + testRunner.setProperty(YandexTranslate.CHARACTER_SET, "UTF-8"); + + testRunner.enqueue("bonjour".getBytes()); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(YandexTranslate.REL_SUCCESS, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(YandexTranslate.REL_SUCCESS).get(0); + + final String outText = new String(out.toByteArray()); + assertEquals("hello", outText); + } + + + @Test + public void testTranslateSingleAttribute() { + testRunner.setProperty(YandexTranslate.KEY, apiKey); + testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); + testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); + testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "false"); + testRunner.setProperty(YandexTranslate.CHARACTER_SET, "UTF-8"); + testRunner.setProperty("translated", "bonjour"); + + testRunner.enqueue(new byte[0]); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(YandexTranslate.REL_SUCCESS, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(YandexTranslate.REL_SUCCESS).get(0); + + assertEquals(0, out.toByteArray().length); + out.assertAttributeEquals("translated", "hello"); + } + + @Test + public void testTranslateMultipleAttributes() { + testRunner.setProperty(YandexTranslate.KEY, apiKey); + testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); + testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); + testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "false"); + testRunner.setProperty(YandexTranslate.CHARACTER_SET, "UTF-8"); + testRunner.setProperty("hello", "bonjour"); + testRunner.setProperty("translate", "traduire"); + testRunner.setProperty("fun", "amusant"); + + testRunner.enqueue(new byte[0]); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(YandexTranslate.REL_SUCCESS, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(YandexTranslate.REL_SUCCESS).get(0); + + assertEquals(0, out.toByteArray().length); + out.assertAttributeEquals("hello", "hello"); + out.assertAttributeEquals("translate", "translate"); + out.assertAttributeEquals("fun", "fun"); + } + + + @Test + public void testTranslateContentAndMultipleAttributes() { + testRunner.setProperty(YandexTranslate.KEY, apiKey); + testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); + testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); + testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "true"); + testRunner.setProperty(YandexTranslate.CHARACTER_SET, "UTF-8"); + testRunner.setProperty("hello", "bonjour"); + testRunner.setProperty("translate", "traduire"); + testRunner.setProperty("fun", "amusant"); + testRunner.setProperty("nifi", "nifi"); + + testRunner.enqueue("ordinateur".getBytes()); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(YandexTranslate.REL_SUCCESS, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(YandexTranslate.REL_SUCCESS).get(0); + + out.assertContentEquals("computer"); + + out.assertAttributeEquals("hello", "hello"); + out.assertAttributeEquals("translate", "translate"); + out.assertAttributeEquals("fun", "fun"); + out.assertAttributeEquals("nifi", "nifi"); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml new file mode 100644 index 0000000000..43573e7b7b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 0.1.0-incubating-SNAPSHOT + + + nifi-language-translation-bundle + pom + + + nifi-yandex-processors + nifi-language-translation-nar + + + + + + com.sun.jersey + jersey-client + ${jersey.version} + + + com.sun.jersey + jersey-json + ${jersey.version} + + + + + From e9cb3b300c6b5e223744925cee1d0c59dd97d29a Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 9 Apr 2015 17:56:52 -0400 Subject: [PATCH 08/16] NIFI-244: Initial import of GetTwitter processor --- nifi/nifi-assembly/NOTICE | 49 + nifi/nifi-assembly/pom.xml | 925 +++++++++--------- nifi/nifi-commons/pom.xml | 1 + .../nifi-social-media-nar/pom.xml | 36 + .../nifi-twitter-processors/.gitignore | 1 + .../nifi-twitter-processors/pom.xml | 60 ++ .../nifi/processors/twitter/GetTwitter.java | 360 +++++++ .../org.apache.nifi.processor.Processor | 16 + .../nifi-social-media-bundle/pom.xml | 33 + nifi/nifi-nar-bundles/pom.xml | 4 + nifi/pom.xml | 24 + 11 files changed, 1060 insertions(+), 449 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-social-media-nar/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/.gitignore create mode 100644 nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java create mode 100644 nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi/nifi-nar-bundles/nifi-social-media-bundle/pom.xml diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE index 8d7db8d7ec..d95e2ffed4 100644 --- a/nifi/nifi-assembly/NOTICE +++ b/nifi/nifi-assembly/NOTICE @@ -501,6 +501,38 @@ The following binary components are provided under the Apache Software License v Apache License Version 2.0 http://www.apache.org/licenses/. (c) Daniel Lemire, http://lemire.me/en/ + (ASLv2) Twitter4J + The following NOTICE information applies: + Copyright 2007 Yusuke Yamamoto + + Twitter4J includes software from JSON.org to parse JSON response from the Twitter API. You can see the license term at http://www.JSON.org/license.html + + (ASLv2) JOAuth + The following NOTICE information applies: + JOAuth + Copyright 2010-2013 Twitter, Inc + + Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0 + + (ASLv2) Hosebird Client + The following NOTICE information applies: + Hosebird Client (hbc) + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0 + + (ASLv2) GeoIP2 Java API + The following NOTICE information applies: + GeoIP2 Java API + This software is Copyright (c) 2013 by MaxMind, Inc. + + This is free software, licensed under the Apache License, Version 2.0. + + (ASLv2) Google HTTP Client Library for Java + The following NOTICE information applies: + Google HTTP Client Library for Java + + This is free software, licensed under the Apache License, Version 2.0. ************************ Common Development and Distribution License 1.1 @@ -540,6 +572,14 @@ The following binary components are provided under the Common Development and Di (CDDL 1.0) JavaServer Pages(TM) API (javax.servlet.jsp:jsp-api:jar:2.1 - http://jsp.java.net) (CDDL 1.0) SR 250 Common Annotations For The JavaTM Platform (javax.annotation:jsr250-api:jar:1.0 - http://jcp.org/aboutJava/communityprocess/final/jsr250/index.html) +************************ +Creative Commons Attribution-ShareAlike 3.0 +************************ + +The following binary components are provided under the Creative Commons Attribution-ShareAlike 3.0. See project link for details. + + (CCAS 3.0) MaxMind DB (https://github.com/maxmind/MaxMind-DB) + ************************ Eclipse Public License 1.0 ************************ @@ -559,6 +599,15 @@ The following binary components are provided under the Mozilla Public License v2 (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:9.6.0-4 - http://www.saxonica.com/) +***************** +Mozilla Public License v1.1 +***************** + +The following binary components are provided under the Mozilla Public License v1.1. See project link for details. + + (MPL 1.1) HAPI Base (ca.uhn.hapi:hapi-base:2.2 - http://http://hl7api.sourceforge.net/) + (MPL 1.1) HAPI Structures (ca.uhn.hapi:hapi-structures-v*:2.2 - http://http://hl7api.sourceforge.net/) + ***************** Public Domain ***************** diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index a26f214c87..13ffba802d 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -1,457 +1,484 @@ - - - 4.0.0 - - org.apache.nifi - nifi - 0.1.0-incubating-SNAPSHOT - - nifi-assembly - pom - This is the assembly Apache NiFi (incubating) - - - - maven-assembly-plugin - - nifi-${project.version} - false - - - - make shared resource - - single - - package - - - src/main/assembly/dependencies.xml - - posix - - - - - - - - - ch.qos.logback - logback-classic - compile - - - org.slf4j - jcl-over-slf4j - compile - - - org.slf4j - jul-to-slf4j - compile - - - org.slf4j - log4j-over-slf4j - compile - - - org.slf4j - slf4j-api - compile - - - org.apache.nifi - nifi-api - - - org.apache.nifi - nifi-runtime - - - org.apache.nifi - nifi-bootstrap - - - org.apache.nifi - nifi-resources - resources - runtime - zip - - - org.apache.nifi - nifi-docs - resources - runtime - zip - - - org.apache.nifi - nifi-framework-nar - nar - - - org.apache.nifi - nifi-provenance-repository-nar - nar - - - org.apache.nifi - nifi-standard-services-api-nar - nar - - - org.apache.nifi - nifi-ssl-context-service-nar - nar - - - org.apache.nifi - nifi-distributed-cache-services-nar - nar - - - org.apache.nifi - nifi-standard-nar - nar - - - org.apache.nifi - nifi-jetty-bundle - nar - - - org.apache.nifi - nifi-update-attribute-nar - nar - - - org.apache.nifi - nifi-hadoop-libraries-nar - nar - - - org.apache.nifi - nifi-hadoop-nar - nar - - - org.apache.nifi - nifi-kafka-nar - nar - + + + 4.0.0 + + org.apache.nifi + nifi + 0.1.0-incubating-SNAPSHOT + + nifi-assembly + pom + This is the assembly Apache NiFi (incubating) + + + + maven-assembly-plugin + + nifi-${project.version} + false + + + + make shared resource + + single + + package + + + src/main/assembly/dependencies.xml + + posix + + + + + + + - org.apache.nifi - nifi-http-context-map-nar - nar - + ch.qos.logback + logback-classic + compile + - org.apache.nifi - nifi-kite-nar - nar - - - - - - 256 - 512 - 128 - 128 - 10m - 10 - - - ${project.version} - true - 10 sec - 500 ms - 30 sec - 10 millis + org.slf4j + jcl-over-slf4j + compile + + + org.slf4j + jul-to-slf4j + compile + + + org.slf4j + log4j-over-slf4j + compile + + + org.slf4j + slf4j-api + compile + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-runtime + + + org.apache.nifi + nifi-bootstrap + + + org.apache.nifi + nifi-resources + resources + runtime + zip + + + org.apache.nifi + nifi-docs + resources + runtime + zip + + + org.apache.nifi + nifi-framework-nar + nar + + + org.apache.nifi + nifi-provenance-repository-nar + nar + + + org.apache.nifi + nifi-standard-services-api-nar + nar + + + org.apache.nifi + nifi-ssl-context-service-nar + nar + + + org.apache.nifi + nifi-distributed-cache-services-nar + nar + + + org.apache.nifi + nifi-standard-nar + nar + + + org.apache.nifi + nifi-jetty-bundle + nar + + + org.apache.nifi + nifi-update-attribute-nar + nar + + + org.apache.nifi + nifi-hadoop-libraries-nar + nar + + + org.apache.nifi + nifi-hadoop-nar + nar + + + org.apache.nifi + nifi-kafka-nar + nar + + + org.apache.nifi + nifi-http-context-map-nar + nar + + + org.apache.nifi + nifi-kite-nar + nar + + + org.apache.nifi + nifi-social-media-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-hl7-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-language-translation-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-geo-nar + 0.1.0-incubating-SNAPSHOT + nar + + - ./conf/flow.xml.gz - ./conf/archive/ - ./conf/authority-providers.xml - ./conf/templates - ./database_repository + + + 256 + 512 + 128 + 128 + 10m + 10 - org.apache.nifi.controller.repository.WriteAheadFlowFileRepository - ./flowfile_repository - 256 - 2 mins - false - org.apache.nifi.controller.FileSystemSwapManager - 20000 - 5 sec - 1 - 5 sec - 4 - - org.apache.nifi.controller.repository.FileSystemRepository - 10 MB - 100 - ./content_repository - - - false - false - - - - - 30 sec - ./lib - ./work/nar/ - ./work/docs/components - - PBEWITHMD5AND256BITAES-CBC-OPENSSL - BC - ;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + + ${project.version} + true + 10 sec + 500 ms + 30 sec + 10 millis - 9990 - - - org.apache.nifi.provenance.PersistentProvenanceRepository - ./provenance_repository - 24 hours - 1 GB - 5 mins - 100 MB - 2 - true - EventType, FlowFileUUID, Filename, ProcessorID - - 500 MB - false - 16 - - - 100000 - - - org.apache.nifi.controller.status.history.VolatileComponentStatusRepository - 288 - 5 mins - - - ./lib - - 8080 - - - ./work/jetty - 200 - - - - - - - - - - - ./conf/authorized-users.xml - 24 hours - file-provider - - - - - - - 5 sec - false - 30 sec - 45 sec - false - - - 500 ms - 3 - 1 sec + ./conf/flow.xml.gz + ./conf/archive/ + ./conf/authority-providers.xml + ./conf/templates + ./database_repository - - false - - - 2 - - - - - false - - - - 10 - 30 sec - 30 sec - 10 - 5 sec - 10 - 0 sec - - - - rpm - - false - - - - - maven-dependency-plugin - - - unpack-shared-resources - - unpack-dependencies - - generate-resources - - ${project.build.directory}/generated-resources - nifi-resources - org.apache.nifi - false - - - - unpack-docs - - unpack-dependencies - - generate-resources - - ${project.build.directory}/generated-docs - nifi-docs - org.apache.nifi - false - - - - - - org.codehaus.mojo - rpm-maven-plugin - - Apache NiFi (incubating) - Apache Nifi (incubating) is dataflow system based on the Flow-Based Programming concepts. - Apache License, Version 2.0 and others (see included LICENSE file) - http://nifi.incubator.apache.org - Utilities - /opt/nifi - - _use_internal_dependency_generator 0 - - 750 - 640 - root - root - - - - build-bin-rpm - - attached-rpm - - - bin - - nifi - - - - /opt/nifi/nifi-${project.version} - - - /opt/nifi/nifi-${project.version} - - - ./LICENSE - - - ./NOTICE - - - ../DISCLAIMER - - - ./README.md - README - - - - - /opt/nifi/nifi-${project.version}/bin - 750 - - - ${project.build.directory}/generated-resources/bin/nifi.sh - nifi.sh - true - - - - - /opt/nifi/nifi-${project.version}/conf - true - - - ${project.build.directory}/generated-resources/conf - true - - - - - /opt/nifi/nifi-${project.version}/lib - - - org.apache.nifi:nifi-bootstrap - org.apache.nifi:nifi-resources - org.apache.nifi:nifi-docs - - - - - /opt/nifi/nifi-${project.version}/lib/bootstrap - - - org.apache.nifi:nifi-bootstrap - - - - - /opt/nifi/nifi-${project.version}/docs - - - ${project.build.directory}/generated-docs - - - - - - - - - - - - + org.apache.nifi.controller.repository.WriteAheadFlowFileRepository + ./flowfile_repository + 256 + 2 mins + false + org.apache.nifi.controller.FileSystemSwapManager + 20000 + 5 sec + 1 + 5 sec + 4 + + org.apache.nifi.controller.repository.FileSystemRepository + 10 MB + 100 + ./content_repository + + + false + false + + + + + 30 sec + ./lib + ./work/nar/ + ./work/docs/components + + PBEWITHMD5AND256BITAES-CBC-OPENSSL + BC + ;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + + 9990 + + + org.apache.nifi.provenance.PersistentProvenanceRepository + ./provenance_repository + 24 hours + 1 GB + 5 mins + 100 MB + 2 + true + EventType, FlowFileUUID, + Filename, ProcessorID + + 500 MB + false + 16 + + + 100000 + + + org.apache.nifi.controller.status.history.VolatileComponentStatusRepository + 288 + 5 mins + + + ./lib + + 8080 + + + ./work/jetty + 200 + + + + + + + + + + + ./conf/authorized-users.xml + 24 hours + file-provider + + + + + + + 5 sec + false + 30 sec + 45 sec + false + + + 500 ms + 3 + 1 sec + + + false + + + 2 + + + + + false + + + + 10 + 30 sec + 30 sec + 10 + 5 sec + 10 + 0 sec + + + + rpm + + false + + + + + maven-dependency-plugin + + + unpack-shared-resources + + unpack-dependencies + + generate-resources + + ${project.build.directory}/generated-resources + nifi-resources + org.apache.nifi + false + + + + unpack-docs + + unpack-dependencies + + generate-resources + + ${project.build.directory}/generated-docs + nifi-docs + org.apache.nifi + false + + + + + + org.codehaus.mojo + rpm-maven-plugin + + Apache NiFi (incubating) + Apache Nifi (incubating) is dataflow system based on + the Flow-Based Programming concepts. + Apache License, Version 2.0 and others (see included + LICENSE file) + http://nifi.incubator.apache.org + Utilities + /opt/nifi + + _use_internal_dependency_generator 0 + + 750 + 640 + root + root + + + + build-bin-rpm + + attached-rpm + + + bin + + nifi + + + + /opt/nifi/nifi-${project.version} + + + /opt/nifi/nifi-${project.version} + + + ./LICENSE + + + ./NOTICE + + + ../DISCLAIMER + + + ./README.md + README + + + + + /opt/nifi/nifi-${project.version}/bin + 750 + + + ${project.build.directory}/generated-resources/bin/nifi.sh + nifi.sh + true + + + + + /opt/nifi/nifi-${project.version}/conf + true + + + ${project.build.directory}/generated-resources/conf + true + + + + + /opt/nifi/nifi-${project.version}/lib + + + org.apache.nifi:nifi-bootstrap + org.apache.nifi:nifi-resources + org.apache.nifi:nifi-docs + + + + + /opt/nifi/nifi-${project.version}/lib/bootstrap + + + org.apache.nifi:nifi-bootstrap + + + + + /opt/nifi/nifi-${project.version}/docs + + + ${project.build.directory}/generated-docs + + + + + + + + + + + + diff --git a/nifi/nifi-commons/pom.xml b/nifi/nifi-commons/pom.xml index 53580549bc..1d2ce46418 100644 --- a/nifi/nifi-commons/pom.xml +++ b/nifi/nifi-commons/pom.xml @@ -36,5 +36,6 @@ nifi-processor-utilities nifi-write-ahead-log nifi-site-to-site-client + nifi-hl7-query-language diff --git a/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-social-media-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-social-media-nar/pom.xml new file mode 100644 index 0000000000..6da74dd68a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-social-media-nar/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-social-media-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-social-media-nar + nar + + + + org.apache.nifi + nifi-twitter-processors + 0.1.0-incubating-SNAPSHOT + + + + diff --git a/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/.gitignore b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/.gitignore new file mode 100644 index 0000000000..b83d22266a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/pom.xml new file mode 100644 index 0000000000..45af0cec5f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-social-media-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-twitter-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + + com.twitter + hbc-twitter4j + 2.2.0 + + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java new file mode 100644 index 0000000000..45b1ae1a13 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/java/org/apache/nifi/processors/twitter/GetTwitter.java @@ -0,0 +1,360 @@ +/* + * 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.twitter; + +import java.io.IOException; +import java.io.OutputStream; +import java.net.MalformedURLException; +import java.nio.charset.StandardCharsets; +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.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.regex.Pattern; + +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +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.AllowableValue; +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.processor.util.StandardValidators; + +import com.twitter.hbc.ClientBuilder; +import com.twitter.hbc.core.Client; +import com.twitter.hbc.core.Constants; +import com.twitter.hbc.core.endpoint.StatusesFilterEndpoint; +import com.twitter.hbc.core.endpoint.StatusesFirehoseEndpoint; +import com.twitter.hbc.core.endpoint.StatusesSampleEndpoint; +import com.twitter.hbc.core.endpoint.StreamingEndpoint; +import com.twitter.hbc.core.event.Event; +import com.twitter.hbc.core.processor.StringDelimitedProcessor; +import com.twitter.hbc.httpclient.auth.Authentication; +import com.twitter.hbc.httpclient.auth.OAuth1; + +@SupportsBatching +@Tags({"twitter", "tweets", "social media", "status", "json"}) +@CapabilityDescription("Pulls status changes from Twitter's streaming API") +@WritesAttribute(attribute="mime.type", description="Sets mime type to application/json") +public class GetTwitter extends AbstractProcessor { + + static final AllowableValue ENDPOINT_SAMPLE = new AllowableValue("Sample Endpoint", "Sample Endpoint", "The endpoint that provides public data, aka a 'garden hose'"); + static final AllowableValue ENDPOINT_FIREHOSE = new AllowableValue("Firehose Endpoint", "Firehose Endpoint", "The endpoint that provides access to all tweets"); + static final AllowableValue ENDPOINT_FILTER = new AllowableValue("Filter Endpoint", "Filter Endpoint", "Endpoint that allows the stream to be filtered by specific terms or User IDs"); + + public static final PropertyDescriptor ENDPOINT = new PropertyDescriptor.Builder() + .name("Twitter Endpoint") + .description("Specifies which endpoint data should be pulled from") + .required(true) + .allowableValues(ENDPOINT_SAMPLE, ENDPOINT_FIREHOSE, ENDPOINT_FILTER) + .defaultValue(ENDPOINT_SAMPLE.getValue()) + .build(); + public static final PropertyDescriptor CONSUMER_KEY = new PropertyDescriptor.Builder() + .name("Consumer Key") + .description("The Consumer Key provided by Twitter") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor CONSUMER_SECRET = new PropertyDescriptor.Builder() + .name("Consumer Secret") + .description("The Consumer Secret provided by Twitter") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder() + .name("Access Token") + .description("The Acces Token provided by Twitter") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor ACCESS_TOKEN_SECRET = new PropertyDescriptor.Builder() + .name("Access Token Secret") + .description("The Access Token Secret provided by Twitter") + .required(true) + .sensitive(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor LANGUAGES = new PropertyDescriptor.Builder() + .name("Languages") + .description("A comma-separated list of languages for which tweets should be fetched") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor FOLLOWING = new PropertyDescriptor.Builder() + .name("IDs to Follow") + .description("A comma-separated list of Twitter User ID's to follow. Ignored unless Endpoint is set to 'Filter Endpoint'.") + .required(false) + .addValidator(new FollowingValidator()) + .build(); + public static final PropertyDescriptor TERMS = new PropertyDescriptor.Builder() + .name("Terms to Filter On") + .description("A comma-separated list of terms to filter on. Ignored unless Endpoint is set to 'Filter Endpoint'. The filter works such that if any term matches, the status update will be retrieved; multiple terms separated by a space function as an 'AND'. I.e., 'it was, hello' will retrieve status updates that have either 'hello' or both 'it' AND 'was'") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All status updates will be routed to this relationship") + .build(); + + private List descriptors; + private Set relationships; + + private final BlockingQueue eventQueue = new LinkedBlockingQueue(1000); + + private volatile Client client; + private volatile BlockingQueue messageQueue; + + @Override + protected void init(final ProcessorInitializationContext context) { + final List descriptors = new ArrayList(); + descriptors.add(ENDPOINT); + descriptors.add(CONSUMER_KEY); + descriptors.add(CONSUMER_SECRET); + descriptors.add(ACCESS_TOKEN); + descriptors.add(ACCESS_TOKEN_SECRET); + descriptors.add(LANGUAGES); + descriptors.add(TERMS); + descriptors.add(FOLLOWING); + this.descriptors = Collections.unmodifiableList(descriptors); + + final Set relationships = new HashSet(); + relationships.add(REL_SUCCESS); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set getRelationships() { + return this.relationships; + } + + @Override + public final List getSupportedPropertyDescriptors() { + return descriptors; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .description("Adds a query parameter with name '" + propertyDescriptorName + "' to the Twitter query") + .required(false) + .dynamic(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final List results = new ArrayList<>(); + final String endpointName = validationContext.getProperty(ENDPOINT).getValue(); + + if ( ENDPOINT_FILTER.getValue().equals(endpointName) ) { + if ( !validationContext.getProperty(TERMS).isSet() && !validationContext.getProperty(FOLLOWING).isSet() ) { + results.add(new ValidationResult.Builder().input("").subject(FOLLOWING.getName()).valid(false).explanation("When using the 'Filter Endpoint', at least one of '" + TERMS.getName() + "' or '" + FOLLOWING.getName() + "' must be set").build()); + } + } + + return results; + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + // if any property is modified, the results are no longer valid. Destroy all messages in teh queue. + messageQueue.clear(); + } + + @OnScheduled + public void onScheduled(final ProcessContext context) throws MalformedURLException { + messageQueue = new LinkedBlockingQueue<>(100000); + + final String endpointName = context.getProperty(ENDPOINT).getValue(); + final Authentication oauth = new OAuth1(context.getProperty(CONSUMER_KEY).getValue(), + context.getProperty(CONSUMER_SECRET).getValue(), + context.getProperty(ACCESS_TOKEN).getValue(), + context.getProperty(ACCESS_TOKEN_SECRET).getValue()); + + final ClientBuilder clientBuilder = new ClientBuilder(); + clientBuilder.name("GetTwitter[id=" + getIdentifier() + "]") + .authentication(oauth) + .eventMessageQueue(eventQueue) + .processor(new StringDelimitedProcessor(messageQueue)); + + final String languageString = context.getProperty(LANGUAGES).getValue(); + final List languages; + if ( languageString == null ) { + languages = null; + } else { + languages = new ArrayList<>(); + for ( final String language : context.getProperty(LANGUAGES).getValue().split(",") ) { + languages.add(language.trim()); + } + } + + final String host; + final StreamingEndpoint streamingEndpoint; + if ( ENDPOINT_SAMPLE.getValue().equals(endpointName) ) { + host = Constants.STREAM_HOST; + final StatusesSampleEndpoint sse = new StatusesSampleEndpoint(); + streamingEndpoint = sse; + if ( languages != null ) { + sse.languages(languages); + } + } else if ( ENDPOINT_FIREHOSE.getValue().equals(endpointName) ) { + host = Constants.STREAM_HOST; + final StatusesFirehoseEndpoint firehoseEndpoint = new StatusesFirehoseEndpoint(); + streamingEndpoint = firehoseEndpoint; + if ( languages != null ) { + firehoseEndpoint.languages(languages); + } + } else if ( ENDPOINT_FILTER.getValue().equals(endpointName) ) { + host = Constants.STREAM_HOST; + final StatusesFilterEndpoint filterEndpoint = new StatusesFilterEndpoint(); + + final String followingString = context.getProperty(FOLLOWING).getValue(); + final List followingIds; + if ( followingString == null ) { + followingIds = Collections.emptyList(); + } else { + followingIds = new ArrayList<>(); + + for ( final String split : followingString.split(",") ) { + final Long id = Long.parseLong(split.trim()); + followingIds.add(id); + } + } + + final String termString = context.getProperty(TERMS).getValue(); + final List terms; + if ( termString == null ) { + terms = Collections.emptyList(); + } else { + terms = new ArrayList<>(); + for ( final String split : termString.split(",") ) { + terms.add(split.trim()); + } + } + + if ( !terms.isEmpty() ) { + filterEndpoint.trackTerms(terms); + } + + if ( !followingIds.isEmpty() ) { + filterEndpoint.followings(followingIds); + } + + if ( languages != null ) { + filterEndpoint.languages(languages); + } + streamingEndpoint = filterEndpoint; + } else { + throw new AssertionError("Endpoint was invalid value: " + endpointName); + } + + clientBuilder.hosts(host).endpoint(streamingEndpoint); + client = clientBuilder.build(); + client.connect(); + } + + @OnStopped + public void shutdownClient() { + if ( client != null ) { + client.stop(); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + final Event event = eventQueue.poll(); + if ( event != null ) { + switch (event.getEventType()) { + case STOPPED_BY_ERROR: + getLogger().error("Received error {}: {} due to {}. Will not attempt to reconnect", new Object[] {event.getEventType(), event.getMessage(), event.getUnderlyingException()}); + break; + case CONNECTION_ERROR: + case HTTP_ERROR: + getLogger().error("Received error {}: {}. Will attempt to reconnect", new Object[] {event.getEventType(), event.getMessage()}); + client.reconnect(); + break; + default: + break; + } + } + + final String tweet = messageQueue.poll(); + if ( tweet == null ) { + context.yield(); + return; + } + + FlowFile flowFile = session.create(); + flowFile = session.write(flowFile, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + out.write(tweet.getBytes(StandardCharsets.UTF_8)); + } + }); + + final Map attributes = new HashMap<>(); + attributes.put(CoreAttributes.MIME_TYPE.key(), "application/json"); + attributes.put(CoreAttributes.FILENAME.key(), flowFile.getAttribute(CoreAttributes.FILENAME.key()) + ".json"); + flowFile = session.putAllAttributes(flowFile, attributes); + + session.transfer(flowFile, REL_SUCCESS); + session.getProvenanceReporter().receive(flowFile, Constants.STREAM_HOST + client.getEndpoint().getURI().toString()); + } + + private static class FollowingValidator implements Validator { + private static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+"); + + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final String[] splits = input.split(","); + for ( final String split : splits ) { + if ( !NUMBER_PATTERN.matcher(split.trim()).matches() ) { + return new ValidationResult.Builder().input(input).subject(subject).valid(false).explanation("Must be comma-separted list of User ID's").build(); + } + } + + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } + + } +} diff --git a/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..9504a1128e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-social-media-bundle/nifi-twitter-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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.processors.twitter.GetTwitter \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-social-media-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-social-media-bundle/pom.xml new file mode 100644 index 0000000000..5aadbce05e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-social-media-bundle/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 0.1.0-incubating-SNAPSHOT + + + nifi-social-media-bundle + pom + + + nifi-twitter-processors + nifi-social-media-nar + + + diff --git a/nifi/nifi-nar-bundles/pom.xml b/nifi/nifi-nar-bundles/pom.xml index e7c122d5e8..50a940721a 100644 --- a/nifi/nifi-nar-bundles/pom.xml +++ b/nifi/nifi-nar-bundles/pom.xml @@ -35,6 +35,10 @@ nifi-update-attribute-bundle nifi-kafka-bundle nifi-kite-bundle + nifi-social-media-bundle + nifi-geo-bundle + nifi-hl7-bundle + nifi-language-translation-bundle diff --git a/nifi/pom.xml b/nifi/pom.xml index 2e2346a6f5..9b8bfb441d 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -798,6 +798,30 @@ 0.1.0-incubating-SNAPSHOT nar + + org.apache.nifi + nifi-social-media-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-hl7-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-language-translation-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-geo-nar + 0.1.0-incubating-SNAPSHOT + nar + org.apache.nifi nifi-properties From b682b6fab543cabeb3a321d6e5cf22f7ce9968c1 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 9 Apr 2015 17:59:02 -0400 Subject: [PATCH 09/16] NIFI-74, NIFI-345, NIFI-495: Fixed several site-to-site related bugs --- .../client/socket/EndpointConnectionPool.java | 43 +++++++++++++++---- .../remote/client/socket/SocketClient.java | 3 +- .../protocol/socket/SocketClientProtocol.java | 8 +++- .../socket/SocketClientTransaction.java | 25 +++++++---- .../nifi/remote/StandardRemoteGroupPort.java | 1 + 5 files changed, 61 insertions(+), 19 deletions(-) diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java index 1a6dfd510a..1b5412cd01 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java @@ -88,6 +88,7 @@ import org.apache.nifi.web.api.dto.ControllerDTO; import org.apache.nifi.web.api.dto.PortDTO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.helpers.MessageFormatter; public class EndpointConnectionPool { public static final long PEER_REFRESH_PERIOD = 60000L; @@ -202,6 +203,28 @@ public class EndpointConnectionPool { }, 5, 5, TimeUnit.SECONDS); } + void warn(final String msg, final Object... args) { + logger.warn(msg, args); + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.WARNING, "Site-to-Site", MessageFormatter.arrayFormat(msg, args).getMessage()); + } + } + + void warn(final String msg, final Throwable t) { + logger.warn(msg, t); + + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.WARNING, "Site-to-Site", msg + ": " + t.toString()); + } + } + + void error(final String msg, final Object... args) { + logger.error(msg, args); + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", MessageFormatter.arrayFormat(msg, args).getMessage()); + } + } + private String getPortIdentifier(final TransferDirection transferDirection) throws IOException { if ( remoteDestination.getIdentifier() != null ) { return remoteDestination.getIdentifier(); @@ -271,6 +294,7 @@ public class EndpointConnectionPool { logger.debug("{} No Connection available for Port {}; creating new Connection", this, portId); protocol = new SocketClientProtocol(); protocol.setDestination(new IdEnrichedRemoteDestination(remoteDestination, portId)); + protocol.setEventReporter(eventReporter); final long penalizationMillis = remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS); try { @@ -312,7 +336,9 @@ public class EndpointConnectionPool { // handle error cases if ( protocol.isDestinationFull() ) { - logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer); + logger.warn("{} {} indicates that port {}'s destination is full; penalizing peer", + this, peer, config.getPortName() == null ? config.getPortIdentifier() : config.getPortName()); + penalize(peer, penalizationMillis); try { peer.close(); @@ -341,7 +367,7 @@ public class EndpointConnectionPool { cleanup(protocol, peer); final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? clusterUrl : peer, e.toString()); - logger.error(message); + error(message); if ( logger.isDebugEnabled() ) { logger.error("", e); } @@ -463,7 +489,7 @@ public class EndpointConnectionPool { peerList = createPeerStatusList(direction); } catch (final Exception e) { final String message = String.format("%s Failed to update list of peers due to %s", this, e.toString()); - logger.warn(message); + warn(message); if ( logger.isDebugEnabled() ) { logger.warn("", e); } @@ -503,7 +529,7 @@ public class EndpointConnectionPool { } private boolean isPenalized(final PeerStatus peerStatus) { - final Long expirationEnd = peerTimeoutExpirations.get(peerStatus); + final Long expirationEnd = peerTimeoutExpirations.get(peerStatus.getPeerDescription()); return (expirationEnd == null ? false : expirationEnd > System.currentTimeMillis() ); } @@ -587,7 +613,7 @@ public class EndpointConnectionPool { clientProtocol.shutdown(peer); } catch (final IOException e) { final String message = String.format("%s Failed to shutdown protocol when updating list of peers due to %s", this, e.toString()); - logger.warn(message); + warn(message); if (logger.isDebugEnabled()) { logger.warn("", e); } @@ -597,7 +623,7 @@ public class EndpointConnectionPool { peer.close(); } catch (final IOException e) { final String message = String.format("%s Failed to close resources when updating list of peers due to %s", this, e.toString()); - logger.warn(message); + warn(message); if (logger.isDebugEnabled()) { logger.warn("", e); } @@ -622,7 +648,8 @@ public class EndpointConnectionPool { } } catch (final IOException e) { - logger.error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString(), e); + error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString()); + logger.error("", e); } } @@ -818,7 +845,7 @@ public class EndpointConnectionPool { peerStatusCache = new PeerStatusCache(statuses); logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", this, statuses.size()); } catch (Exception e) { - logger.warn("{} Unable to refresh Remote Group's peers due to {}", this, e); + warn("{} Unable to refresh Remote Group's peers due to {}", this, e); if (logger.isDebugEnabled()) { logger.warn("", e); } diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java index ed54ccb99b..4aab3f7f8e 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java @@ -84,6 +84,7 @@ public class SocketClient implements SiteToSiteClient { logger.debug("Unable to resolve port [{}] to an identifier", portName); } else { logger.debug("Resolved port [{}] to identifier [{}]", portName, portId); + this.portIdentifier = portId; } return portId; @@ -136,7 +137,7 @@ public class SocketClient implements SiteToSiteClient { connectionState.getPeer(), connectionState.getCodec(), direction); } catch (final Throwable t) { pool.terminate(connectionState); - throw t; + throw new IOException("Unable to create Transaction to communicate with " + connectionState.getPeer(), t); } // Wrap the transaction in a new one that will return the EndpointConnectionState back to the pool whenever diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java index c3275eab00..83c5305054 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import org.apache.nifi.events.EventReporter; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.ProcessContext; @@ -75,6 +76,7 @@ public class SocketClientProtocol implements ClientProtocol { private int batchCount; private long batchSize; private long batchMillis; + private EventReporter eventReporter; private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds @@ -93,6 +95,10 @@ public class SocketClientProtocol implements ClientProtocol { this.batchMillis = millis; } + public void setEventReporter(final EventReporter eventReporter) { + this.eventReporter = eventReporter; + } + public void setDestination(final RemoteDestination destination) { this.destination = destination; this.useCompression = destination.isUseCompression(); @@ -272,7 +278,7 @@ public class SocketClientProtocol implements ClientProtocol { } return new SocketClientTransaction(versionNegotiator.getVersion(), destination.getIdentifier(), peer, codec, - direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS)); + direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS), eventReporter); } diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java index a1ce07e450..e69104f201 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java @@ -27,6 +27,7 @@ import java.util.zip.CRC32; import java.util.zip.CheckedInputStream; import java.util.zip.CheckedOutputStream; +import org.apache.nifi.events.EventReporter; import org.apache.nifi.remote.Communicant; import org.apache.nifi.remote.Peer; import org.apache.nifi.remote.Transaction; @@ -39,6 +40,7 @@ import org.apache.nifi.remote.io.CompressionOutputStream; import org.apache.nifi.remote.protocol.DataPacket; import org.apache.nifi.remote.protocol.RequestType; import org.apache.nifi.remote.util.StandardDataPacket; +import org.apache.nifi.reporting.Severity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +58,7 @@ public class SocketClientTransaction implements Transaction { private final Peer peer; private final int penaltyMillis; private final String destinationId; + private final EventReporter eventReporter; private boolean dataAvailable = false; private int transfers = 0; @@ -63,7 +66,7 @@ public class SocketClientTransaction implements Transaction { private TransactionState state; SocketClientTransaction(final int protocolVersion, final String destinationId, final Peer peer, final FlowFileCodec codec, - final TransferDirection direction, final boolean useCompression, final int penaltyMillis) throws IOException { + final TransferDirection direction, final boolean useCompression, final int penaltyMillis, final EventReporter eventReporter) throws IOException { this.protocolVersion = protocolVersion; this.destinationId = destinationId; this.peer = peer; @@ -74,6 +77,7 @@ public class SocketClientTransaction implements Transaction { this.compress = useCompression; this.state = TransactionState.TRANSACTION_STARTED; this.penaltyMillis = penaltyMillis; + this.eventReporter = eventReporter; initialize(); } @@ -116,11 +120,11 @@ public class SocketClientTransaction implements Transaction { try { try { if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) { - throw new IllegalStateException("Cannot receive data because Transaction State is " + state); + throw new IllegalStateException("Cannot receive data from " + peer + " because Transaction State is " + state); } if ( direction == TransferDirection.SEND ) { - throw new IllegalStateException("Attempting to receive data but started a SEND Transaction"); + throw new IllegalStateException("Attempting to receive data from " + peer + " but started a SEND Transaction"); } // if we already know there's no data, just return null @@ -142,7 +146,7 @@ public class SocketClientTransaction implements Transaction { this.dataAvailable = false; break; default: - throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode); + throw new ProtocolException("Got unexpected response from " + peer + " when asking for data: " + dataAvailableCode); } } @@ -184,11 +188,11 @@ public class SocketClientTransaction implements Transaction { try { try { if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) { - throw new IllegalStateException("Cannot send data because Transaction State is " + state); + throw new IllegalStateException("Cannot send data to " + peer + " because Transaction State is " + state); } if ( direction == TransferDirection.RECEIVE ) { - throw new IllegalStateException("Attempting to send data but started a RECEIVE Transaction"); + throw new IllegalStateException("Attempting to send data to " + peer + " but started a RECEIVE Transaction"); } if ( transfers > 0 ) { @@ -242,7 +246,7 @@ public class SocketClientTransaction implements Transaction { try { try { if ( state != TransactionState.TRANSACTION_CONFIRMED ) { - throw new IllegalStateException("Cannot complete transaction because state is " + state + + throw new IllegalStateException("Cannot complete transaction with " + peer + " because state is " + state + "; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED); } @@ -272,7 +276,7 @@ public class SocketClientTransaction implements Transaction { peer.penalize(destinationId, penaltyMillis); backoff = true; } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) { - throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse); + throw new ProtocolException("After sending data to " + peer + ", expected TRANSACTION_FINISHED response but got " + transactionResponse); } state = TransactionState.TRANSACTION_COMPLETED; @@ -324,7 +328,10 @@ public class SocketClientTransaction implements Transaction { try { confirmTransactionResponse = Response.read(dis); } catch (final IOException ioe) { - logger.error("Failed to receive response code from {} when expected confirmation of transaction", peer); + logger.error("Failed to receive response code from {} when expecting confirmation of transaction", peer); + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", "Failed to receive response code from " + peer + " when expecting confirmation of transaction"); + } throw ioe; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java index 69ba0fd4f8..eec6ed58b5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java @@ -183,6 +183,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort { remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); return; } catch (final IOException e) { + context.yield(); final String message = String.format("%s failed to communicate with %s due to %s", this, url, e.toString()); logger.error(message); if ( logger.isDebugEnabled() ) { From 10d002c78a7fbd31669d840199f3138d715d6189 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 08:40:22 -0400 Subject: [PATCH 10/16] NIFI-244: Fixed copy/paste error that called out website as http://http://... instead of just http://... --- nifi/nifi-assembly/NOTICE | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE index d95e2ffed4..9da37ae104 100644 --- a/nifi/nifi-assembly/NOTICE +++ b/nifi/nifi-assembly/NOTICE @@ -605,8 +605,8 @@ Mozilla Public License v1.1 The following binary components are provided under the Mozilla Public License v1.1. See project link for details. - (MPL 1.1) HAPI Base (ca.uhn.hapi:hapi-base:2.2 - http://http://hl7api.sourceforge.net/) - (MPL 1.1) HAPI Structures (ca.uhn.hapi:hapi-structures-v*:2.2 - http://http://hl7api.sourceforge.net/) + (MPL 1.1) HAPI Base (ca.uhn.hapi:hapi-base:2.2 - http://hl7api.sourceforge.net/) + (MPL 1.1) HAPI Structures (ca.uhn.hapi:hapi-structures-v*:2.2 - http://hl7api.sourceforge.net/) ***************** Public Domain From 512ac9c704e155461dac0c0c905ecccaba645c3b Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 08:42:43 -0400 Subject: [PATCH 11/16] NIFI-504: Removed erroneous copyright notice --- .../java/org/apache/nifi/processors/maxmind/DatabaseReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java index 796a7aff5c..734f093e45 100644 --- a/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java +++ b/nifi/nifi-nar-bundles/nifi-geo-bundle/nifi-geo-processors/src/main/java/org/apache/nifi/processors/maxmind/DatabaseReader.java @@ -46,7 +46,7 @@ import com.maxmind.geoip2.model.IspResponse; * This class was copied from * https://raw.githubusercontent.com/maxmind/GeoIP2-java/master/src/main/java/com/maxmind/geoip2/DatabaseReader.java * It is written by Maxmind and it is available under Apache Software License V2 - * Copyright (c) 2013 by MaxMind, Inc. + * * The modification we're making to the code below is to stop using exceptions for * mainline flow control. Specifically we don't want to throw an exception * simply because an address was not found. From e18c0a7d267460293a7e5a0189b223607212354f Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 09:37:53 -0400 Subject: [PATCH 12/16] NIFI-503: Removed dependencies on commons-lang3 and commons-io --- .../nifi-framework/nifi-documentation/pom.xml | 8 -------- .../html/HtmlDocumentationWriter.java | 15 ++++++++++++--- .../html/HtmlProcessorDocumentationWriter.java | 16 +++++++++++----- .../html/HtmlDocumentationWriterTest.java | 8 ++++++++ 4 files changed, 31 insertions(+), 16 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml index e522d30ff7..5995f5c327 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml @@ -32,14 +32,6 @@ org.apache.nifi nifi-properties - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - org.apache.nifi nifi-processor-utils diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index 34b1327513..243aaa35ae 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -26,7 +26,6 @@ import javax.xml.stream.XMLOutputFactory; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; -import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperties; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; @@ -212,13 +211,23 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeEndElement(); xmlStreamWriter.writeStartElement("p"); if (tags != null) { - final String tagString = StringUtils.join(tags.value(), ", "); + final String tagString = join(tags.value(), ", "); xmlStreamWriter.writeCharacters(tagString); } else { xmlStreamWriter.writeCharacters("None."); } xmlStreamWriter.writeEndElement(); - + } + + static String join(final String[] toJoin, final String delimiter) { + final StringBuilder sb = new StringBuilder(); + for (int i=0; i < toJoin.length; i++) { + sb.append(toJoin[i]); + if ( i < toJoin.length - 1 ) { + sb.append(delimiter); + } + } + return sb.toString(); } /** diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java index bc355f7712..c253a4d8f2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java @@ -23,7 +23,6 @@ import java.util.List; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; -import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.behavior.ReadsAttributes; @@ -67,6 +66,13 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { handleWritesAttributes(xmlStreamWriter, processor); } + private String defaultIfBlank(final String test, final String defaultValue) { + if ( test == null || test.trim().isEmpty() ) { + return defaultValue; + } + return test; + } + /** * Writes out just the attributes that are being read in a table form. * @@ -91,10 +97,10 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { for (ReadsAttribute attribute : attributesRead) { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + defaultIfBlank(attribute.attribute(), "Not Specified")); // TODO allow for HTML characters here. writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); + defaultIfBlank(attribute.description(), "Not Specified")); xmlStreamWriter.writeEndElement(); } @@ -129,10 +135,10 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { for (WritesAttribute attribute : attributesRead) { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + defaultIfBlank(attribute.attribute(), "Not Specified")); // TODO allow for HTML characters here. writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); + defaultIfBlank(attribute.description(), "Not Specified")); xmlStreamWriter.writeEndElement(); } xmlStreamWriter.writeEndElement(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java index 9d7926e858..90ff09f19c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java @@ -30,9 +30,17 @@ import org.apache.nifi.reporting.ReportingTask; import org.junit.Test; import static org.apache.nifi.documentation.html.XmlValidator.assertContains; +import static org.junit.Assert.assertEquals; public class HtmlDocumentationWriterTest { + @Test + public void testJoin() { + assertEquals("a, b, c", HtmlDocumentationWriter.join(new String[] {"a", "b", "c"}, ", ")); + assertEquals("a, b", HtmlDocumentationWriter.join(new String[] {"a", "b"}, ", ")); + assertEquals("a", HtmlDocumentationWriter.join(new String[] {"a"}, ", ")); + } + @Test public void testDocumentControllerService() throws InitializationException, IOException { From 2c6b2a56640df57cb816b84b39f061d0d5271bce Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 11:06:12 -0400 Subject: [PATCH 13/16] NIFI-507: Make default not to yield for ports and funnels --- .../nifi/controller/tasks/ContinuallyRunConnectableTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index b5b60c9bf8..5ecd22e657 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -107,6 +107,6 @@ public class ContinuallyRunConnectableTask implements Callable { return true; } - return true; + return false; // do not yield } } From 3515f2c12fedb8cd1e88bb8793b75a31317038b3 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 11:20:23 -0400 Subject: [PATCH 14/16] Revert "NIFI-488: Redirect nifi output streams and redirect bootstrap log messages to file" This reverts commit abd279c1e01e3aea3332e6b378701554722317e6. --- .../org/apache/nifi/bootstrap/RunNiFi.java | 131 +++--------------- .../src/main/resources/conf/bootstrap.conf | 5 - 2 files changed, 17 insertions(+), 119 deletions(-) diff --git a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java index d25df97025..28a9b71028 100644 --- a/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java +++ b/nifi/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java @@ -45,10 +45,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.logging.FileHandler; +import java.util.logging.ConsoleHandler; import java.util.logging.Handler; import java.util.logging.Level; -import java.util.logging.SimpleFormatter; /** @@ -93,82 +92,16 @@ public class RunNiFi { private final java.util.logging.Logger logger; - public RunNiFi(final File bootstrapConfigFile, final boolean verbose) throws IOException { + public RunNiFi(final File bootstrapConfigFile, final boolean verbose) { this.bootstrapConfigFile = bootstrapConfigFile; logger = java.util.logging.Logger.getLogger("Bootstrap"); - - final Properties bootstrapProps = new Properties(); - try (final InputStream configIn = new FileInputStream(bootstrapConfigFile)) { - bootstrapProps.load(configIn); - } - - String logFilename = bootstrapProps.getProperty("bootstrap.log.file"); - if ( logFilename == null ) { - logFilename = "./logs/bootstrap.log"; - } - - File logFile = new File(logFilename); - if ( !logFile.isAbsolute() ) { - final File workDir = getDefaultWorkingDirectory(); - logFile = new File(workDir, logFilename); - } - - final File logFileDir = logFile.getParentFile(); - final Handler fileHandler; - if ( logFileDir.exists() || logFileDir.mkdirs() ) { - final int maxSize = getIntProp(bootstrapProps, "bootstrap.log.max.bytes", 1024 * 1024 * 10); // 10 MB - final int numFiles = getIntProp(bootstrapProps, "bootstrap.log.count", 10); - - fileHandler = new FileHandler(logFile.getAbsolutePath(), maxSize, numFiles, true); - fileHandler.setFormatter(new SimpleFormatter()); - logger.addHandler(fileHandler); - } else { - fileHandler = null; - logger.severe("Could not create log file directory " + logFileDir + ". Will not log bootstrap info to file or redirect NiFi standard out to file"); - } - if ( verbose ) { logger.info("Enabling Verbose Output"); logger.setLevel(Level.FINE); - - for ( final Handler handler : logger.getHandlers() ) { - handler.setLevel(Level.FINE); - } - } - } - - - private File getLogFile() throws IOException { - final Properties bootstrapProps = new Properties(); - try (final InputStream configIn = new FileInputStream(bootstrapConfigFile)) { - bootstrapProps.load(configIn); - } - - String logFilename = bootstrapProps.getProperty("bootstrap.log.file"); - if ( logFilename == null ) { - logFilename = "./logs/bootstrap.log"; - } - - File logFile = new File(logFilename); - if ( !logFile.isAbsolute() ) { - final File workDir = getDefaultWorkingDirectory(); - logFile = new File(workDir, logFilename); - } - - return logFile; - } - - private static int getIntProp(final Properties properties, final String name, final int defaultValue) { - String propVal = properties.getProperty(name); - if ( propVal == null || propVal.trim().isEmpty() ) { - return defaultValue; - } - - try { - return Integer.parseInt(propVal.trim()); - } catch (final NumberFormatException nfe) { - throw new NumberFormatException("Expected bootstrap property '" + name + "' to be an integer but found value: " + propVal); + final Handler handler = new ConsoleHandler(); + handler.setLevel(Level.FINE); + logger.addHandler(handler); } } @@ -648,35 +581,6 @@ public class RunNiFi { } } - private void redirectOutput(final Process process) { - redirectStreamToLogs(process.getInputStream()); - redirectStreamToLogs(process.getErrorStream()); - } - - private void redirectStreamToLogs(final InputStream in) { - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { - String line; - while ((line = reader.readLine()) != null) { - logger.info(line); - } - } catch (IOException e) { - logger.warning("Failed to read output of NiFi console: " + e); - } - } - }); - t.setDaemon(true); - t.start(); - } - - private File getDefaultWorkingDirectory() { - final File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile(); - final File binDir = bootstrapConfigAbsoluteFile.getParentFile(); - return binDir.getParentFile(); - } - @SuppressWarnings({ "rawtypes", "unchecked" }) public void start(final boolean monitor) throws IOException, InterruptedException { final Integer port = getCurrentPort(); @@ -686,6 +590,7 @@ public class RunNiFi { } final ProcessBuilder builder = new ProcessBuilder(); + if ( !bootstrapConfigFile.exists() ) { throw new FileNotFoundException(bootstrapConfigFile.getAbsolutePath()); } @@ -699,16 +604,17 @@ public class RunNiFi { props.putAll( (Map) properties ); final String specifiedWorkingDir = props.get("working.dir"); - final File workingDir = getDefaultWorkingDirectory(); - if ( specifiedWorkingDir == null ) { - builder.directory(workingDir); - } else { + if ( specifiedWorkingDir != null ) { builder.directory(new File(specifiedWorkingDir)); } + + final File bootstrapConfigAbsoluteFile = bootstrapConfigFile.getAbsoluteFile(); + final File binDir = bootstrapConfigAbsoluteFile.getParentFile(); + final File workingDir = binDir.getParentFile(); - final File logDir = getLogFile().getParentFile(); - builder.redirectError(new File(logDir, "nifi.err")); - builder.redirectOutput(new File(logDir, "nifi.out")); + if ( specifiedWorkingDir == null ) { + builder.directory(workingDir); + } final String libFilename = replaceNull(props.get("lib.dir"), "./lib").trim(); File libDir = getFile(libFilename, workingDir); @@ -832,15 +738,14 @@ public class RunNiFi { try { gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown); } catch (final NumberFormatException nfe) { - throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); + throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); } if ( gracefulShutdownSeconds < 0 ) { - throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); + throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File " + bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer"); } Process process = builder.start(); - redirectOutput(process); Long pid = getPid(process); if ( pid != null ) { nifiPid = pid; @@ -871,8 +776,7 @@ public class RunNiFi { if (autoRestartNiFi) { logger.warning("Apache NiFi appears to have died. Restarting..."); process = builder.start(); - redirectOutput(process); - + pid = getPid(process); if ( pid != null ) { nifiPid = pid; @@ -898,7 +802,6 @@ public class RunNiFi { } } else { final Process process = builder.start(); - redirectOutput(process); final Long pid = getPid(process); if ( pid != null ) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf index e87bde2673..c1536d829f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf @@ -21,11 +21,6 @@ java=java # Username to use when running NiFi. This value will be ignored on Windows. run.as= -# Bootstrap logger info -bootstrap.log.file=logs/nifi-bootstrap.log -bootstrap.log.max.bytes=10485760 -bootstrap.log.count=10 - # Configure where NiFi's lib and conf directories live lib.dir=./lib conf.dir=./conf From c83c7afd9898e1032984a81d3792b174222930df Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 14:25:53 -0400 Subject: [PATCH 15/16] NIFI-506: Fixed unit tests and resources, and added RAT exclusions --- .../nifi-hl7-query-language/pom.xml | 10 ++++ .../nifi/hl7/query/antlr/HL7QueryLexer.g | 17 ++++++ .../nifi/hl7/query/antlr/HL7QueryParser.g | 17 ++++++ .../apache/nifi/hl7/query/TestHL7Query.java | 52 ++-------------- .../src/test/resources/hyperglycemia | 8 +-- .../src/test/resources/hypoglycemia | 8 +-- .../src/test/resources/metabolic-panel | 23 ------- .../resources/unsolicited-vaccine-update-long | 16 ----- .../unsolicited-vaccine-update-short | 4 -- .../src/test/resources/vaccine-query | 3 - .../src/test/resources/vaers-message-long | 60 ------------------- .../nifi-hl7-processors/pom.xml | 14 +++++ .../hl7/TestExtractHL7Attributes.java | 2 +- .../src/test/resources/1.hl7 | 16 ----- .../src/test/resources/hypoglycemia.hl7 | 10 ++-- 15 files changed, 77 insertions(+), 183 deletions(-) delete mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel delete mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long delete mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short delete mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query delete mode 100644 nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long delete mode 100644 nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/pom.xml b/nifi/nifi-commons/nifi-hl7-query-language/pom.xml index 447a88b84d..9d0003fcec 100644 --- a/nifi/nifi-commons/nifi-hl7-query-language/pom.xml +++ b/nifi/nifi-commons/nifi-hl7-query-language/pom.xml @@ -48,6 +48,16 @@ + + org.apache.rat + apache-rat-plugin + + + src/test/resources/hypoglycemia + src/test/resources/hyperglycemia + + + diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g index 7fe3386b4d..478028b9f7 100644 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g @@ -1,3 +1,20 @@ +/* + * 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. + */ + lexer grammar HL7QueryLexer; @header { diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g index 4d8d13c59a..f051872eba 100644 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g @@ -1,3 +1,20 @@ +/* + * 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. + */ + parser grammar HL7QueryParser; options { diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java b/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java index fbe4a8d285..596440c380 100644 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; import java.nio.file.Files; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; @@ -32,8 +31,6 @@ import java.util.Map; import org.apache.nifi.hl7.hapi.HapiMessage; import org.apache.nifi.hl7.model.HL7Field; import org.apache.nifi.hl7.model.HL7Message; -import org.apache.nifi.hl7.query.HL7Query; -import org.apache.nifi.hl7.query.QueryResult; import org.junit.Test; import ca.uhn.hl7v2.DefaultHapiContext; @@ -99,7 +96,7 @@ public class TestHL7Query { @Test public void testSelectMessage() throws HL7Exception, IOException { final HL7Query query = HL7Query.compile("SELECT MESSAGE"); - final HL7Message msg = createMessage(new File("src/test/resources/vaers-message-long")); + final HL7Message msg = createMessage(new File("src/test/resources/hypoglycemia")); final QueryResult result = query.evaluate(msg); assertTrue(result.isMatch()); final List labels = result.getLabels(); @@ -114,7 +111,7 @@ public class TestHL7Query { @SuppressWarnings({ "unchecked", "rawtypes" }) public void testSelectField() throws HL7Exception, IOException { final HL7Query query = HL7Query.compile("SELECT PID.5"); - final HL7Message msg = createMessage(new File("src/test/resources/unsolicited-vaccine-update-short")); + final HL7Message msg = createMessage(new File("src/test/resources/hypoglycemia")); final QueryResult result = query.evaluate(msg); assertTrue(result.isMatch()); final List labels = result.getLabels(); @@ -126,7 +123,7 @@ public class TestHL7Query { final List nameList = (List) names; assertEquals(1, nameList.size()); final HL7Field nameField = (HL7Field) nameList.get(0); - assertEquals("KENNEDY^JOHN^FITZGERALD^JR", nameField.getValue()); + assertEquals("SMITH^JOHN", nameField.getValue()); } @Test @@ -134,8 +131,8 @@ public class TestHL7Query { final String query = "DECLARE result AS REQUIRED OBX SELECT result WHERE result.7 != 'N' AND result.1 = 1"; final HL7Query hl7Query = HL7Query.compile(query); - final QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/vaers-message-long"))); - assertFalse( result.isMatch() ); + final QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); } @@ -224,18 +221,6 @@ public class TestHL7Query { QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); assertTrue( result.isMatch() ); assertEquals(1, result.getHitCount()); - - hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT result WHERE result.1 = 1"); - HL7Message msg = createMessage(new File("src/test/resources/vaers-message-long")); - result = hl7Query.evaluate(msg); - assertTrue( result.isMatch() ); - assertEquals(9, result.getHitCount()); - - hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT result WHERE result.1 = 1 AND result.3.1.1 = '30961-7'"); - result = hl7Query.evaluate(msg); - assertTrue( result.isMatch() ); - assertEquals(1, result.getHitCount()); - } @Test @@ -288,10 +273,6 @@ public class TestHL7Query { hl7Query = HL7Query.compile("SELECT MESSAGE WHERE OBX IS NULL"); result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); assertFalse( result.isMatch() ); - - hl7Query = HL7Query.compile("SELECT MESSAGE WHERE NK1.1 = '1' AND NK1.8 IS NULL"); - result = hl7Query.evaluate(createMessage(new File("src/test/resources/unsolicited-vaccine-update-long"))); - assertTrue( result.isMatch() ); } @@ -312,14 +293,6 @@ public class TestHL7Query { hl7Query = HL7Query.compile("SELECT MESSAGE WHERE OBX NOT NULL"); result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); assertTrue( result.isMatch() ); - - hl7Query = HL7Query.compile("SELECT MESSAGE WHERE NK1.1 = '1' AND NK1.33 NOT NULL"); - result = hl7Query.evaluate(createMessage(new File("src/test/resources/unsolicited-vaccine-update-long"))); - assertTrue( result.isMatch() ); - - hl7Query = HL7Query.compile("SELECT MESSAGE WHERE NK1.1 = 1 AND NK1.33 NOT NULL"); - result = hl7Query.evaluate(createMessage(new File("src/test/resources/unsolicited-vaccine-update-long"))); - assertTrue( result.isMatch() ); } private HL7Message createMessage(final File file) throws HL7Exception, IOException { @@ -334,19 +307,4 @@ public class TestHL7Query { return new HapiMessage(message); } - @Test - @SuppressWarnings("unused") - public void createMessage() throws IOException, HL7Exception { - final byte[] bytes = Files.readAllBytes(Paths.get("src/test/resources/vaers-message-long")); - final String msgText = new String(bytes, "UTF-8"); - - final HapiContext hapiContext = new DefaultHapiContext(); - hapiContext.setValidationContext(ValidationContextFactory.noValidation()); - - final PipeParser parser = hapiContext.getPipeParser(); - final Message message = parser.parse(msgText); - - final HL7Message hl7Msg = new HapiMessage(message); - } - } diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia index dc44b89ac2..9c6888c598 100644 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia @@ -1,5 +1,5 @@ -MSH|^~\&|CERNER||PriorityHealth||||ORU^R01|Q479004375T431430612|P|2.3| -PID|||001677980||SMITH^CURTIS||19680219|M||||||||||929645156318|123456789| +MSH|^~\&|XXXXXXXX||HealthProvider||||ORU^R01|Q1111111111111111111|P|2.3| +PID|||111111111||SMITH^JOHN||19700100|M||||||||||111111111111|123456789| PD1||||1234567890^LAST^FIRST^M^^^^^NPI| -OBR|1|341856649^HNAM_ORDERID|000002006326002362|648088^Basic Metabolic Panel|||20061122151600|||||||||1620^Hooker^Robert^L||||||20061122154733|||F|||||||||||20061122140000| -OBX|1|NM|GLU^Glucose Lvl|159|mg/dL|65-99^65^99|H|||F|||20061122154733| \ No newline at end of file +OBR|1|341856649^HNAM_ORDERID|000000000000000000|648088^Basic Metabolic Panel|||20150101000000|||||||||1620^Johnson^Corey^A||||||20150101000000|||F|||||||||||20150101000000| +OBX|1|NM|GLU^Glucose Lvl|159|mg/dL|65-99^65^99|H|||F|||20150102000000| \ No newline at end of file diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia index 02e8967296..d7bdeece10 100644 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia @@ -1,5 +1,5 @@ -MSH|^~\&|CERNER||PriorityHealth||||ORU^R01|Q479004375T431430612|P|2.3| -PID|||001677980||SMITH^CURTIS||19680219|M||||||||||929645156318|123456789| +MSH|^~\&|XXXXXXXX||HealthProvider||||ORU^R01|Q1111111111111111111|P|2.3| +PID|||111111111||SMITH^JOHN||19700100|M||||||||||111111111111|123456789| PD1||||1234567890^LAST^FIRST^M^^^^^NPI| -OBR|1|341856649^HNAM_ORDERID|000002006326002362|648088^Basic Metabolic Panel|||20061122151600|||||||||1620^Hooker^Robert^L||||||20061122154733|||F|||||||||||20061122140000| -OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20061122154733| \ No newline at end of file +OBR|1|341856649^HNAM_ORDERID|000000000000000000|648088^Basic Metabolic Panel|||20150101000000|||||||||1620^Johnson^Corey^A||||||20150101000000|||F|||||||||||20150101000000| +OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20150102000000| \ No newline at end of file diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel deleted file mode 100644 index c62fc45a32..0000000000 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/metabolic-panel +++ /dev/null @@ -1,23 +0,0 @@ -MSH|^~\&|Lab1^1234^CLIA|^1234^CLIA|ELR^2.16.840.1.113883.19.3.2^ISO|SPH^2.16.840.1.113883.19.3.2^ISO|20110410140502-0500||ORU^R01^ORU_R01|1234567890|P^T|2.5.1|||NE|NE|USA||||USELR1.0^^2.16.840.1.114222.4.10.3^ISO -SFT|1|Level Seven Healthcare Software, Inc.^L^^^^&2.16.840.1.113883.19.4.6^ISO^XX^^^1234|1.2|An Lab System|56734||20080817 -PID|1||36363636^^^MPI&2.16.840.1.113883.19.3.2.1&ISO^MR^A&2.16.840.1.113883.19.3.2.1&ISO~444333333^^^&2.16.840.1.113883.4.1^IS O^SS||Everyman^Adam^A^^^^L^^^^^^^BS|Mum^Martha^M^^^^M|19800602|M||2106-3^White^CDCREC^^^^04/24/2007|2222 Home Street^^Ann Arbor^MI^99999^USA^H||^PRN^PH^^1^555^5552004|^WPN^PH^^1^955^5551009|eng^English^ISO6392^^^^3/29/2007|M^Married^HL70002^^^^2.5.1||||||N^Not Hispanic or Latino^HL70189^^^^2.5.1||||||||N|||200808151000-0700| Reliable^2.16.840.1.113883.19.3.1^ISO -ORC|RE|23456^EHR^2.16.840.1.113883.19.3.2.3^ISO|9700123^Lab^2.16.840.1.113883.19.3.1.6^ISO|||||||||1234^Admit^Alan^A^III^Dr^^^&2.16.840.1.113883.19.4.6^ISO^L^^^EI^&2.16.840.1.113883.19.4.6^ISO^^^^^^^^MD||^WPN^PH^^1^555^5551005|||||||Level Seven Healthcare, Inc.^L^^^^&2.16.840.1.113883.19.4.6^ISO^XX^^^1234|1005 Healthcare Drive^^Ann Arbor^MI^99999^USA^B|^WPN^PH^^1^555^5553001|4444 Healthcare Drive^Suite 123^Ann Arbor^MI^99999^USA^B -OBR|1|23456^EHR^2.16.840.1.113883.19.3.2.3^ISO|9700123^Lab^2.16.840.1.113883.19.3.1.6^ISO|24323-8^Comprehensive metabolic 2000 panel in Serum or Plasma^LN^3436442^Metaboloic Panel 2000, Comprehensive^99USI|||201104101130-0500||||||angina|||1234^Admit^Alan^A^III^Dr^^^&2.16.840.1.113883.19.4.6^ISO^L^^^EI^&2.16.840.1.113883.19.4.6^ISO^^^^^^^^MD|^WPN^PH^^1^555^5551005|||||201104101405-0500|||F||||||413^Angina pectoris^I9CDX^^^^07/09/2008|1235&Slide&Stan&S&&Dr&MD&&DOC&2.16.840.1.113883.19.4.6&ISO -OBX|1|NM|17861-6^Calcium [Mass/volume] in Serum or Plasma^LN||27.3|mg/dL^milligrams per deciliter^UCUM|8.7-10.7|HH|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|2|NM|3094-0^Urea nitrogen [Mass/volume] in Serum of Plasma^LN||15|mg/dL^milligrams per deciliter^UCUM|6 to 23|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|3|NM|2160-0^Creatinine [Mass/volume] in Serum or Plasma^LN||1.8|mg/dL^milligrams per deciliter^UCUM|0.7 to 1.2|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|4|NM|3097-3^Urea nitrogen/Creatinine [Mass ratio] in Serum or Plasma^LN||15||6 to 25|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|5|NM|2885-2^Protein [Mass/volume] in Serum or Plasma^LN||8.9|gm/dL^grams per deciliter^UCUM|6.3 to 8.2|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|6|NM|1751-7^Albumin [Mass/volume] in Serum or Plasma^LN||5.7|gm/dL^grams per deciliter^UCUM|3.5 to 5.0|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|7|NM|2336-6^Globulin [Mass/volume] in Serum or Plasma^LN||4.7|gm/dL^grams per deciliter^UCUM|2.2 to 4.2|H|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|8|NM|1759-0^Albumin/Globulin [Mass ratio] in Serum or Plasma^LN||1.7||0.8 to 2.0|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|9|NM|1975-2^Bilirubin.total [Mass/volume] in Serum or Plasma^LN||0.7|mg/dL^milligrams per deciliter^UCUM|0.3 to 1.9|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|10|NM|2345-7^Glucose [Mass/volume] in Serum or Plasma^LN||55|mg/dL^milligrams per deciliter^UCUM|60 to 109|L|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|11|NM|6768-6^Alkaline phosphatase [Enzymatic activity/volume] in Serum or Plasma^LN||64|U/L^units per liter^UCUM|32 to 110|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|12|NM|1920-8^Aspartate aminotransferase [Enzymatic activity/volume] in Serum or Plasma^LN||6|U/L^units per liter^UCUM|6 to 18|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|13|NM|1742-6^Alanine aminotransferase [Enzymatic activity/volume] in Serum or Plasma^LN||10|U/L^units per liter^UCUM|5 to 35|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|14|NM|2951-2^Sodium [Moles/volume] in Serum or Plasma^LN||140|mmol/L^millimoles per liter^UCUM|137 to 147|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|15|NM|2823-3^Potassium [Moles/volume] in Serum or Plasma^LN||4.5|mmol/L^millimoles per liter^UCUM|3.4 to 5.3|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|16|NM|2075-0^Chloride [Moles/volume] in Serum or Plasma^LN||99|mmol/L^millimoles per liter^UCUM|99 to 108|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -OBX|17|NM|2028-9^Carbon dioxide, total [Moles/volume] in Serum or Plasma^LN||27|mmol/L^millimoles per liter^UCUM|22 to 29|N|||F|||201104101130-0500|||||201104101325-0500||||GHH Lab^L^^^^CLIA&2.16.840.1.113883.19.4.6&ISO^XX^^^1236|3434 Industrial Loop^^Ann Arbor^MI^99999^USA^B|9876543^Slide^Stan^S^^^^^NPPES&2.16.840.1.113883.19.4.6&ISO^L^^^NPI -SPM|1|23456&EHR&2.16.840.1.113883.19.3.2.3&ISO^9700122&Lab&2.16.840.1.113883.19.3.1.6&ISO||119364003^Serum specimen^SCT^^^^20080131|||||||||||||201104101130-0500|201104101130-0500 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long deleted file mode 100644 index 8edd3fd82e..0000000000 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-long +++ /dev/null @@ -1,16 +0,0 @@ -MSH|^~\&||MA0000||GA0000|19970901||VXU^V04|19970522MA53|T|2.3.1|||AL -PID|||1234^^^^SR^~1234-12^^^^LR^~3872^^^^MR~221345671^^^^SS^~430078856^^^^MA^ ||KENNEDY^JOHN^FITZGERALD^JR^^^L|BOUVIER^^^^^^M|19900607|M|KENNEDY^BABY BOY^^^^^^ B|W^WHITE^NY8 RACE CODES^W^WHITE^HL70005|123 MAIN ST^APT 3B^LEXINGTON^MA^00210^ ^M^MSA CODE^MA034~345 ELM ST^^BOSTON^MA^00314^^BLD~^^^^^^BR^^MA002| |(617) 555-1212 ^PRN^PH^^^617^5551212^^||EN^ENGLISH^HL70296^^^|||||||WN^NOT HISPANIC^LOCAL CODE SET^NH^NOT OF HISPANIC ORIGIN^HL70189|CHILDREN=S HOSPITAL -PD1|||CHILDREN=S HOSPITAL^^1234^^^^XX~LEXINGTON CLINIC^^1234A^^^^FI|12345^CARE^ PRIMARY^^^DR^MD^^^L^^^DN|||||||03^REMINDER/RECALL - NO CALLS^HL70215|Y -NK1|1|KENNEDY^JACQUELINE^LEE|32^MOTHER^HL70063||||||||||||||||||||||||||||||898666725^^^^SS -NK1|2|KENNEDY^JOHN^FITZGERALD|33^FATHER^HL70063||||||||||||||||||||||||||||||822546618^^^^SS -PV1||R|||||||||||||||A|||V02^19900607~H02^19900607 -RXA|0|1|19900607|19900607|08^HEPB-PEDIATRIC/ADOLESCENT^CVX^90744^HEPB-PEDATRIC/ADOLESCENT^CPT|.5|ML^^ISO+||03^HISTORICAL INFORMATION - FROM PARENT=S WRITTEN RECORD^NIP0001|^JONES^LISA|^^^CHILDREN=S HOSPITAL||5|MCG^^ISO+|MRK12345| 199206|MSD^MERCK^MVX -RXA|0|4|19910907|19910907|50^DTAP-HIB^CVX^90721^DTAP-HIB^CPT|.5|ML^^ISO+||00^NEW IMMUNIZATION RECORD^NIP0001|1234567890^SMITH^SALLY^S^^^^^^^^^VEI~1234567891 ^O=BRIAN^ROBERT^A^^DR^MD^^^^^^OEI|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^ BOSTON^MA||||W46932777|199208|PMC^PASTEUR MERIEUX CONNAUGHT^MVX|||CP|A| 19910907120030 -RXR|IM^INTRAMUSCULAR^HL70162|LA^LEFT ARM^HL70163 -RXA|0|1|19910907|19910907|03^MMR^CVX|.5|ML^^ISO+|||1234567890^SMITH^SALLY^S^^^^^^^^^VEI~1234567891^O=BRIAN^ROBERT^A^^DR^MD^^^^^^OEI|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^BOSTON^MA||||W2348796456|19920731|MSD^MERCK^MVX -RXR|SC^SUBCUTANEOUS^HL70162|LA^LEFT ARM^HL70163 -RXA|0|5|19950520|19950520|20^DTAP^CVX|.5|ML^^ISO+|||1234567891^O=BRIAN^ROBERT^A^^DR|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^BOSTON^MA||||W22532806|19950705|PMC^ PASTEUR MERIEUX CONNAUGHT^MVX -RXR|IM^INTRAMUSCULAR^HL70162|LA^LEFT ARM^HL70163 -NTE|PATIENT DEVELOPED HIGH FEVER APPROX 3 HRS AFTER VACCINE INJECTION -RXA|0|2|19950520|19950520|03^MMR^CVX|.5|ML^^ISO+|||1234567891^O=BRIAN^ROBERT^A^^DR|^^^CHILD HEALTHCARE CLINIC^^^^^101 MAIN STREET^^BOSTON^MA||||W2341234567|19950630| MSD^MERCK^MVX -RXR|SC^SUBCUTANEOUS^HL70162|LA^LEFT ARM^HL70163 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short deleted file mode 100644 index c3e7cf0ff2..0000000000 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/unsolicited-vaccine-update-short +++ /dev/null @@ -1,4 +0,0 @@ -MSH|^~\&|||||||VXU^V04|19970522MA53|P|2.3.1 -PID|||221345671^^^^SS||KENNEDY^JOHN^FITZGERALD^JR|BOUVIER^^^^^^M|19900607|M|||^^^^MA^^^BLD -NK1|1|KENNEDY^JACQUELINE^LEE|32^MOTHER^HL70063 -RXA|0|1|19900607|19900607|08^HEPB-PEDIATRIC/ADOLESCENT^CVX|.5|ML^^ISO+||||||||MRK12345||MSD^MERCK^MVX diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query deleted file mode 100644 index 1bd47b285e..0000000000 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaccine-query +++ /dev/null @@ -1,3 +0,0 @@ -MSH|^~\&||GA0000||MA0000|199705221605||VXQ^V01|19970522GA40|T|2.3.1|||AL -QRD|199705221605|R|I|19970522GA05|||25^RD|^KENNEDY^JOHN^FITZGERALD^JR|VXI^VACCINE INFORMATION^HL70048|^SIIS -QRF|MA0000||||256946789~19900607~MA~MA99999999~88888888~KENNEDY^JACQUELINE^LEE~BOUVIER~898666725~KENNEDY^JOHN^FITZGERALD~822546618 diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long deleted file mode 100644 index 888b04cc0f..0000000000 --- a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/vaers-message-long +++ /dev/null @@ -1,60 +0,0 @@ -MSH|^~\&||GA0000||VAERS PROCESSOR|20010331605||ORU^R01|20010422GA03|T|2.3.1|||AL| -PID|||1234^^^^SR~1234-12^^^^LR~00725^^^^MR||Doe^John^Fitzgerald^JR^^^L||20001007|M||2106-3^White^HL70005|123 Peachtree St^APT 3B^Atlanta^GA^30210^^M^^GA067||(678) 555-1212^^PRN| -NK1|1|Jones^Jane^Lee^^RN|VAB^Vaccine administered by (Name)^HL70063| -NK1|2|Jones^Jane^Lee^^RN|FVP^Form completed by (Name)-Vaccine provider^HL70063|101 Main Street^^Atlanta^GA^38765^^O^^GA121||(404) 554-9097^^WPN| -ORC|CN|||||||||||1234567^Welby^Marcus^J^Jr^Dr.^MD^L|||||||||Peachtree Clinic|101 Main Street^^Atlanta^GA^38765^^O^^GA121|(404) 554-9097^^WPN|101 Main Street^^Atlanta^GA^38765^^O^^GA121| -OBR|1|||^CDC VAERS-1 (FDA) Report|||20010316| -OBX|1|NM|21612-7^Reported Patient Age^LN||05|mo^month^ANSI| -OBX|1|TS|30947-6^Date form completed^LN||20010316| -OBX|2|FT|30948-4^Vaccination adverse events and treatment, if any^LN|1|fever of 106F, with vomiting, seizures, persistent crying lasting over 3 hours, loss of appetite| -OBX|3|CE|30949-2^Vaccination adverse event outcome^LN|1|E^required emergency room/doctor visit^NIP005| -OBX|4|CE|30949-2^Vaccination adverse event outcome^LN|1|H^required hospitalization^NIP005| -OBX|5|NM|30950-0^Number of days hospitalized due to vaccination adverse event^LN|1|02|d^day^ANSI| -OBX|6|CE|30951-8^Patient recovered^LN||Y^Yes^ HL70239| -OBX|7|TS|30952-6^Date of vaccination^LN||20010216| -OBX|8|TS|30953-4^Adverse event onset date and time^LN||200102180900| -OBX|9|FT|30954-2^Relevant diagnostic tests/lab data^LN||Electrolytes, CBC, Blood culture| -OBR|2|||30955-9^All vaccines given on date listed in #10^LN| -OBX|1|CE30955-9&30956-7^Vaccine type^LN|1|08^HepB-Adolescent/pediatric^CVX| -OBX|2|CE|30955-9&30957-5^Manufacturer^LN|1|MSD^Merck^MVX| -OBX|3|ST|30955-9&30959-1^Lot number^LN|1|MRK12345| -OBX|4|CE|30955-9&30958-3^ Route^LN|1|IM^Intramuscular ^HL70162| -OBX|5|CE|30955-9&31034-2^Site^LN|1|LA^Left arm^ HL70163| -OBX|6|NM|30955-9&30960-9^Number of previous doses^LN|1|01I -OBX|7|CE|CE|30955-9&30956-7^Vaccine type^LN|2|50^DTaP-Hib^CVX| -OBX|8|CE|30955-9&30957-5^ Manufacturer^LN|2|WAL^Wyeth_Ayerst^MVX| -OBX|9|ST|30955-9&30959-1^Lot number^LN|2|W46932777| -OBX|10|CE|30955-9&30958-3^ Route^LN|2|IM^Intramuscular^HL70162| -OBX|11|CE|30955-9&31034-2^Site^LN|2|LA^Left arm^HL70163| -OBX|12|NM|30955-9&30960-9^Number of previous doses^LN|2|01| -OBR|3|||30961-7^Any other vaccinations within 4 weeks prior to the date listed in #10| -OBX|1|CE|30961-7&30956-7^Vaccine type^LN|1|10^IPV^CVX| -OBX|2|CE|30961-7&30957-5^Manufacturer^LN|1|PMC^Aventis Pasteur ^MVX| -OBX|3|ST|30961-7&30959-1^Lot number^LN|1|PMC123456| -OBX|4|CE|30961-7&30958-3^Route^LN|1|SC^Subcutaneaous^HL70162| -OBX|5|CE|30961-7&31034-2^Site^LN|1|LA^Left arm^HL70163| -OBX|6|NM|30961-7&30960-9^Number of previous doses^LN|1|01| -OBX|7|TS|30961-7&31035-9^date given^LN|1|20001216| -OBX|8|CE|30962-^Vaccinated at^LN||PVT^Private doctor�s office/hospital^NIP009| -OBX|9|CE|30963-3^Vaccine purchased with^LN||PBF^Public funds^NIP008| -OBX|10|FT|30964-1^Other medications^LN||None| -OBX|11|FT|30965-8^Illness at time of vaccination (specify)^LN||None| -OBX|12|FT|30966-6^Pre-existing physician diagnosed allergies, birth defects, medical conditions^LN||Past conditions convulsions| -OBX|13|CE|30967-4^Was adverse event reported previously^LN||N^no^NIP009| -OBR|4||30968-2^Adverse event following prior vaccination in patient^LN| -OBX|1|TX|30968-2&30971-6^Adverse event^LN||None| -OBR|5||30969-0^Adverse event following prior vaccination in brother^LN| -OBX|1|TX||30969-0&30971-6^Adverse event^LN||vomiting, fever, otitis media| -OBX|2|NM||30969-0&30972-4^Onset age^LN||04|mo^month^ANSI| -OBX|3|CE||30969-0&30956-7^Vaccine Type ^LN||10^IPV^CVX| -OBX|4|NM||30969-0&30973-2^Dose number in series^LN||02| -OBR|6|||30970-8^Adverse event following prior vaccination in sister^LN| -OBX|1|TX|30970-8&30971-6^Adverse event^LN||None| -OBR|7||^For children 5 and under| -OBX|1|NM|8339-4^Body weight at birth^LN||82|oz^ounces^ANSI| -OBX|2|NM|30974-0^Number of brothers and sisters^LN||2| -OBR|8|||^Only for reports submitted by manufacturer/immunization project| -OBX|1|ST|30975-7^Mfr./Imm. Proj. report no.^LN||12345678| -OBX|2|TS|30976-5^Date received by manufacturer/immunization project^LN||12345678| -OBX|3|CE|30977-3^15 day report^LN||N^No^HL70136| -OBX|4|CE|30978-1^Report type^LN||IN^Initial^NIP010| diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml index 2a0c90905b..5f29cd8d27 100644 --- a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/pom.xml @@ -25,6 +25,20 @@ nifi-hl7-processors jar + + + + org.apache.rat + apache-rat-plugin + + + src/test/resources/hypoglycemia.hl7 + + + + + + org.apache.nifi diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java index f56628811e..c1568105b8 100644 --- a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/java/org/apache/nifi/processors/hl7/TestExtractHL7Attributes.java @@ -34,7 +34,7 @@ public class TestExtractHL7Attributes { public void testExtract() throws IOException { System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", "DEBUG"); final TestRunner runner = TestRunners.newTestRunner(ExtractHL7Attributes.class); - runner.enqueue(Paths.get("src/test/resources/1.hl7")); + runner.enqueue(Paths.get("src/test/resources/hypoglycemia.hl7")); runner.run(); runner.assertAllFlowFilesTransferred(ExtractHL7Attributes.REL_SUCCESS, 1); diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 deleted file mode 100644 index bf2b8a581d..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/1.hl7 +++ /dev/null @@ -1,16 +0,0 @@ -MSH|^~`&|ECG REPORTING|ROCHESTER|ERIS|ROCHESTER|20110621050440||ORU^R01|20110621050440|P|2.1 -PID|||999999999||TEST^PATIENT||18450101|F -OBR|||211088491|0^ADULT^ROCHECG|||20110620170631|||||||||M999999^^^^^^^RACFID||||||20110621060232||EC|F|||||||M999999^LASTNAME MD^FIRSTNAME^^^^^RACFID -OBX||ST|93000.2^VENTRICULAR RATE EKG/MIN^CPT4|1|52|/SEC -OBX||ST|93000.4^PR INTERVAL(MSEC)^CPT4|2|208|MSEC -OBX||ST|93000.5^QRS - INTERVAL(MSEC)^CPT4|3|88|MSEC -OBX||ST|93000.6^QT - INTERVAL(MSEC)^CPT4|4|466|MSEC -OBX||ST|93000&PTL^PHYSICAL TEST LOCATION^CPT4|5|STMA -OBX||ST|93000&PTR^PHYSICAL TEST ROOM^CPT4|6|04254 -OBX||CE|93000.17^^CPT4|7|21&101^Sinus bradycardia`T`with 1st degree A-V block^MEIECG -OBX||CE|93000.17^^CPT4|8|1687^Otherwise normal ECG^MEIECG -OBX||CE|93000&CMP^^CPT4|9|1301^When compared with ECG of^MEIECG -OBX||TS|93000&CMD^EKG COMPARISON DATE^CPT4|10|201106171659 -OBX||CE|93000&CMP^^CPT4|11|1305^No significant change was found^MEIECG -OBX||TX|93000.48^EKG COMMENT^CPT4|12|9917^LASTNAME MD^FIRSTNAME -OBX||FT|93000^ECG 12-LEAD^CPT4|13|{\rtf1\ansi \deff1\deflang1033\ {\fonttbl{\f1\fmodern\fcharset0 Courier;}{\f2\fmodern\fcharset0 Courier;}} \pard\plain \f1\fs18\par 20Jun2011 17:06\par VENTRICULAR RATE 52\par Sinus bradycardia with 1st degree A-V block\par Otherwise normal ECG\par When compared with ECG of 17-JUN-2011 16:59,\par No significant change was found\par 47507`S`'LASTNAME MD`S`'FIRSTNAME \par } \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 index 02e8967296..aaff9fc7e7 100644 --- a/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 +++ b/nifi/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/test/resources/hypoglycemia.hl7 @@ -1,5 +1,5 @@ -MSH|^~\&|CERNER||PriorityHealth||||ORU^R01|Q479004375T431430612|P|2.3| -PID|||001677980||SMITH^CURTIS||19680219|M||||||||||929645156318|123456789| -PD1||||1234567890^LAST^FIRST^M^^^^^NPI| -OBR|1|341856649^HNAM_ORDERID|000002006326002362|648088^Basic Metabolic Panel|||20061122151600|||||||||1620^Hooker^Robert^L||||||20061122154733|||F|||||||||||20061122140000| -OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20061122154733| \ No newline at end of file +MSH|^~\&|XXXXXXXX||HealthProvider||||ORU^R01|Q1111111111111111111|P|2.3| +PID|||111111111||SMITH^JOHN||19700100|M||||||||||111111111111|123456789| +PD1||||1234567890^LAST^FIRST^M^^^^^NPI| +OBR|1|341856649^HNAM_ORDERID|000000000000000000|648088^Basic Metabolic Panel|||20150101000000|||||||||1620^Johnson^Corey^A||||||20150101000000|||F|||||||||||20150101000000| +OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20150102000000| \ No newline at end of file From 1963c27409d99d2a05a68a8ed5ededd076bf0da7 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 10 Apr 2015 14:51:27 -0400 Subject: [PATCH 16/16] NIFI-505: Added more legit unit tests --- .../processors/yandex/YandexTranslate.java | 32 +++-- .../yandex/TestYandexTranslate.java | 135 ++++++++++++++---- 2 files changed, 130 insertions(+), 37 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java index a5eecc6238..bd83a8a1d7 100644 --- a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/main/java/org/apache/nifi/processors/yandex/YandexTranslate.java @@ -92,7 +92,7 @@ public class YandexTranslate extends AbstractProcessor { .name("Input Language") .description("The language of incoming data") .required(true) - .defaultValue("sp") + .defaultValue("es") .expressionLanguageSupported(true) .addValidator(new LanguageNameValidator()) .build(); @@ -213,6 +213,24 @@ public class YandexTranslate extends AbstractProcessor { } } + + protected WebResource.Builder prepareResource(final String key, final List text, final String sourceLanguage, final String destLanguage) { + WebResource webResource = client.resource(URL); + + final MultivaluedMap paramMap = new MultivaluedMapImpl(); + paramMap.put("text", text); + paramMap.add("key", key); + paramMap.add("lang", sourceLanguage + "-" + destLanguage); + + WebResource.Builder builder = webResource + .accept(MediaType.APPLICATION_JSON) + .type(MediaType.APPLICATION_FORM_URLENCODED); + builder = builder.entity(paramMap); + + return builder; + } + + @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { FlowFile flowFile = session.get(); @@ -247,17 +265,7 @@ public class YandexTranslate extends AbstractProcessor { textValues.add(content); } - WebResource webResource = client.resource(URL); - - final MultivaluedMap paramMap = new MultivaluedMapImpl(); - paramMap.put("text", textValues); - paramMap.add("key", key); - paramMap.add("lang", sourceLanguage + "-" + targetLanguage); - - WebResource.Builder builder = webResource - .accept(MediaType.APPLICATION_JSON) - .type(MediaType.APPLICATION_FORM_URLENCODED); - builder = builder.entity(paramMap); + final WebResource.Builder builder = prepareResource(key, textValues, sourceLanguage, targetLanguage); final ClientResponse response; try { diff --git a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java index 4f82049353..7b38b8ba1e 100644 --- a/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java +++ b/nifi/nifi-nar-bundles/nifi-language-translation-bundle/nifi-yandex-processors/src/test/java/org/apache/nifi/processors/yandex/TestYandexTranslate.java @@ -18,41 +18,100 @@ package org.apache.nifi.processors.yandex; import static org.junit.Assert.assertEquals; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Properties; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; -import org.apache.nifi.processors.yandex.YandexTranslate; +import javax.ws.rs.core.Response.Status.Family; +import javax.ws.rs.core.Response.StatusType; + +import org.apache.nifi.processors.yandex.model.Translation; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; -import org.junit.Before; -import org.junit.Ignore; +import org.junit.BeforeClass; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.api.client.WebResource.Builder; -@Ignore("For local testing only; requires local file to be populated with Yandex API Key") public class TestYandexTranslate { - private TestRunner testRunner; - private String apiKey; + private static final Map translations = new HashMap<>(); + + @BeforeClass + public static void setupTranslationMap() { + translations.put("bonjour", "hello"); + translations.put("traduire", "translate"); + translations.put("amusant", "fun"); + translations.put("ordinateur", "computer"); + } + + private TestRunner createTestRunner(final int statusCode) { + return TestRunners.newTestRunner(new YandexTranslate() { + @Override + protected Builder prepareResource(final String key, final List text, final String sourceLanguage, final String destLanguage) { + final WebResource.Builder builder = Mockito.mock(WebResource.Builder.class); + + Mockito.doAnswer(new Answer() { + @Override + public ClientResponse answer(final InvocationOnMock invocation) throws Throwable { + final ClientResponse response = Mockito.mock(ClientResponse.class); - @Before - public void init() throws IOException { - testRunner = TestRunners.newTestRunner(YandexTranslate.class); - - final Properties properties = new Properties(); - try (final InputStream in = new FileInputStream(new File("C:/dev/notes/yandex-info.txt"))) { - properties.load(in); - } - apiKey = properties.getProperty("api_key").trim(); + final StatusType statusType = new StatusType() { + @Override + public int getStatusCode() { + return statusCode; + } + + @Override + public String getReasonPhrase() { + return String.valueOf(statusCode); + } + + @Override + public Family getFamily() { + return statusCode == 200 ? Family.SUCCESSFUL : Family.SERVER_ERROR; + } + }; + + Mockito.when(response.getStatus()).thenReturn(statusCode); + Mockito.when(response.getStatusInfo()).thenReturn(statusType); + + if ( statusCode == 200 ) { + final Translation translation = new Translation(); + translation.setCode(statusCode); + translation.setLang(destLanguage); + + final List translationList = new ArrayList<>(); + for ( final String original : text ) { + final String translated = translations.get(original); + translationList.add(translated == null ? original : translated); + } + + translation.setText(translationList); + + Mockito.when(response.getEntity(Translation.class)).thenReturn(translation); + } + + return response; + } + }).when(builder).post(ClientResponse.class); + return builder; + } + }); } - + @Test public void testTranslateContent() { - testRunner.setProperty(YandexTranslate.KEY, apiKey); + final TestRunner testRunner = createTestRunner(200); + testRunner.setProperty(YandexTranslate.KEY, "a"); testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "true"); @@ -71,7 +130,9 @@ public class TestYandexTranslate { @Test public void testTranslateSingleAttribute() { - testRunner.setProperty(YandexTranslate.KEY, apiKey); + final TestRunner testRunner = createTestRunner(200); + + testRunner.setProperty(YandexTranslate.KEY, "A"); testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "false"); @@ -90,7 +151,9 @@ public class TestYandexTranslate { @Test public void testTranslateMultipleAttributes() { - testRunner.setProperty(YandexTranslate.KEY, apiKey); + final TestRunner testRunner = createTestRunner(200); + + testRunner.setProperty(YandexTranslate.KEY, "A"); testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "false"); @@ -114,7 +177,9 @@ public class TestYandexTranslate { @Test public void testTranslateContentAndMultipleAttributes() { - testRunner.setProperty(YandexTranslate.KEY, apiKey); + final TestRunner testRunner = createTestRunner(200); + + testRunner.setProperty(YandexTranslate.KEY, "A"); testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "true"); @@ -137,5 +202,25 @@ public class TestYandexTranslate { out.assertAttributeEquals("fun", "fun"); out.assertAttributeEquals("nifi", "nifi"); } + + @Test + public void testFailureResponse() { + final TestRunner testRunner = createTestRunner(403); + + testRunner.setProperty(YandexTranslate.KEY, "A"); + testRunner.setProperty(YandexTranslate.SOURCE_LANGUAGE, "fr"); + testRunner.setProperty(YandexTranslate.TARGET_LANGUAGE, "en"); + testRunner.setProperty(YandexTranslate.TRANSLATE_CONTENT, "true"); + testRunner.setProperty(YandexTranslate.CHARACTER_SET, "UTF-8"); + testRunner.setProperty("hello", "bonjour"); + testRunner.setProperty("translate", "traduire"); + testRunner.setProperty("fun", "amusant"); + testRunner.setProperty("nifi", "nifi"); + + testRunner.enqueue("ordinateur".getBytes()); + testRunner.run(); + + testRunner.assertAllFlowFilesTransferred(YandexTranslate.REL_TRANSLATION_FAILED, 1); + } }