mirror of https://github.com/apache/nifi.git
NIFI-271 checkpoint push because there are so many changes. Long way to go but got through dto library
This commit is contained in:
parent
8a296aacc9
commit
9faaef8cfa
|
@ -198,6 +198,13 @@
|
|||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-checkstyle-plugin</artifactId>
|
||||
<version>2.15</version>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.puppycrawl.tools</groupId>
|
||||
<artifactId>checkstyle</artifactId>
|
||||
<version>6.5</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
|
@ -310,7 +317,6 @@
|
|||
<property name="allowSamelineMultipleAnnotations" value="true" />
|
||||
</module>
|
||||
<module name="NonEmptyAtclauseDescription" />
|
||||
<module name="JavadocTagContinuationIndentation" />
|
||||
<module name="JavadocMethod">
|
||||
<property name="allowMissingJavadoc" value="true" />
|
||||
<property name="allowMissingParamTags" value="true" />
|
||||
|
@ -325,15 +331,8 @@
|
|||
</checkstyleRules>
|
||||
<violationSeverity>warning</violationSeverity>
|
||||
<includeTestSourceDirectory>true</includeTestSourceDirectory>
|
||||
<excludes>**/HelpMojo.java</excludes>
|
||||
<excludes>**/HelpMojo.java,**/generated-sources</excludes>
|
||||
</configuration>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.puppycrawl.tools</groupId>
|
||||
<artifactId>checkstyle</artifactId>
|
||||
<version>6.3</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>check-style</id>
|
||||
|
@ -379,7 +378,7 @@
|
|||
<plugin>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-nar-maven-plugin</artifactId>
|
||||
<version>1.0.0-incubating</version>
|
||||
<version>1.0.1-incubating-SNAPSHOT</version>
|
||||
<extensions>true</extensions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
|
|
|
@ -27,66 +27,66 @@ import java.util.Arrays;
|
|||
|
||||
import org.apache.nifi.bootstrap.exception.InvalidCommandException;
|
||||
|
||||
|
||||
public class BootstrapCodec {
|
||||
private final RunNiFi runner;
|
||||
private final BufferedReader reader;
|
||||
private final BufferedWriter writer;
|
||||
|
||||
public BootstrapCodec(final RunNiFi runner, final InputStream in, final OutputStream out) {
|
||||
this.runner = runner;
|
||||
this.reader = new BufferedReader(new InputStreamReader(in));
|
||||
this.writer = new BufferedWriter(new OutputStreamWriter(out));
|
||||
}
|
||||
|
||||
public void communicate() throws IOException {
|
||||
final String line = reader.readLine();
|
||||
final String[] splits = line.split(" ");
|
||||
if ( splits.length < 0 ) {
|
||||
throw new IOException("Received invalid command from NiFi: " + line);
|
||||
}
|
||||
|
||||
final String cmd = splits[0];
|
||||
final String[] args;
|
||||
if ( splits.length == 1 ) {
|
||||
args = new String[0];
|
||||
} else {
|
||||
args = Arrays.copyOfRange(splits, 1, splits.length);
|
||||
}
|
||||
|
||||
try {
|
||||
processRequest(cmd, args);
|
||||
} catch (final InvalidCommandException ice) {
|
||||
throw new IOException("Received invalid command from NiFi: " + line + " : " + ice.getMessage() == null ? "" : "Details: " + ice.toString());
|
||||
}
|
||||
}
|
||||
|
||||
private void processRequest(final String cmd, final String[] args) throws InvalidCommandException, IOException {
|
||||
switch (cmd) {
|
||||
case "PORT": {
|
||||
if ( args.length != 2 ) {
|
||||
throw new InvalidCommandException();
|
||||
}
|
||||
|
||||
final int port;
|
||||
try {
|
||||
port = Integer.parseInt( args[0] );
|
||||
} catch (final NumberFormatException nfe) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
if ( port < 1 || port > 65535 ) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
private final RunNiFi runner;
|
||||
private final BufferedReader reader;
|
||||
private final BufferedWriter writer;
|
||||
|
||||
final String secretKey = args[1];
|
||||
|
||||
runner.setNiFiCommandControlPort(port, secretKey);
|
||||
writer.write("OK");
|
||||
writer.newLine();
|
||||
writer.flush();
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
public BootstrapCodec(final RunNiFi runner, final InputStream in, final OutputStream out) {
|
||||
this.runner = runner;
|
||||
this.reader = new BufferedReader(new InputStreamReader(in));
|
||||
this.writer = new BufferedWriter(new OutputStreamWriter(out));
|
||||
}
|
||||
|
||||
public void communicate() throws IOException {
|
||||
final String line = reader.readLine();
|
||||
final String[] splits = line.split(" ");
|
||||
if (splits.length < 0) {
|
||||
throw new IOException("Received invalid command from NiFi: " + line);
|
||||
}
|
||||
|
||||
final String cmd = splits[0];
|
||||
final String[] args;
|
||||
if (splits.length == 1) {
|
||||
args = new String[0];
|
||||
} else {
|
||||
args = Arrays.copyOfRange(splits, 1, splits.length);
|
||||
}
|
||||
|
||||
try {
|
||||
processRequest(cmd, args);
|
||||
} catch (final InvalidCommandException ice) {
|
||||
throw new IOException("Received invalid command from NiFi: " + line + " : " + ice.getMessage() == null ? "" : "Details: " + ice.toString());
|
||||
}
|
||||
}
|
||||
|
||||
private void processRequest(final String cmd, final String[] args) throws InvalidCommandException, IOException {
|
||||
switch (cmd) {
|
||||
case "PORT": {
|
||||
if (args.length != 2) {
|
||||
throw new InvalidCommandException();
|
||||
}
|
||||
|
||||
final int port;
|
||||
try {
|
||||
port = Integer.parseInt(args[0]);
|
||||
} catch (final NumberFormatException nfe) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
if (port < 1 || port > 65535) {
|
||||
throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
|
||||
}
|
||||
|
||||
final String secretKey = args[1];
|
||||
|
||||
runner.setNiFiCommandControlPort(port, secretKey);
|
||||
writer.write("OK");
|
||||
writer.newLine();
|
||||
writer.flush();
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,101 +28,103 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.nifi.bootstrap.util.LimitingInputStream;
|
||||
|
||||
public class NiFiListener {
|
||||
private ServerSocket serverSocket;
|
||||
private volatile Listener listener;
|
||||
|
||||
int start(final RunNiFi runner) throws IOException {
|
||||
serverSocket = new ServerSocket();
|
||||
serverSocket.bind(new InetSocketAddress("localhost", 0));
|
||||
|
||||
final int localPort = serverSocket.getLocalPort();
|
||||
listener = new Listener(serverSocket, runner);
|
||||
final Thread listenThread = new Thread(listener);
|
||||
listenThread.setName("Listen to NiFi");
|
||||
listenThread.start();
|
||||
return localPort;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
final Listener listener = this.listener;
|
||||
if ( listener == null ) {
|
||||
return;
|
||||
}
|
||||
|
||||
listener.stop();
|
||||
}
|
||||
|
||||
private class Listener implements Runnable {
|
||||
private final ServerSocket serverSocket;
|
||||
private final ExecutorService executor;
|
||||
private final RunNiFi runner;
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
public Listener(final ServerSocket serverSocket, final RunNiFi runner) {
|
||||
this.serverSocket = serverSocket;
|
||||
this.executor = Executors.newFixedThreadPool(2);
|
||||
this.runner = runner;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
stopped = true;
|
||||
|
||||
executor.shutdown();
|
||||
try {
|
||||
executor.awaitTermination(3, TimeUnit.SECONDS);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
|
||||
serverSocket.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!serverSocket.isClosed()) {
|
||||
try {
|
||||
if ( stopped ) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Socket socket;
|
||||
try {
|
||||
socket = serverSocket.accept();
|
||||
} catch (final IOException ioe) {
|
||||
if ( stopped ) {
|
||||
return;
|
||||
}
|
||||
|
||||
throw ioe;
|
||||
}
|
||||
|
||||
executor.submit(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
// we want to ensure that we don't try to read data from an InputStream directly
|
||||
// by a BufferedReader because any user on the system could open a socket and send
|
||||
// a multi-gigabyte file without any new lines in order to crash the Bootstrap,
|
||||
// which in turn may cause the Shutdown Hook to shutdown NiFi.
|
||||
// So we will limit the amount of data to read to 4 KB
|
||||
final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096);
|
||||
final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream());
|
||||
codec.communicate();
|
||||
} catch (final Throwable t) {
|
||||
System.out.println("Failed to communicate with NiFi due to " + t);
|
||||
t.printStackTrace();
|
||||
} finally {
|
||||
try {
|
||||
socket.close();
|
||||
} catch (final IOException ioe) {
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (final Throwable t) {
|
||||
System.err.println("Failed to receive information from NiFi due to " + t);
|
||||
t.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ServerSocket serverSocket;
|
||||
private volatile Listener listener;
|
||||
|
||||
int start(final RunNiFi runner) throws IOException {
|
||||
serverSocket = new ServerSocket();
|
||||
serverSocket.bind(new InetSocketAddress("localhost", 0));
|
||||
|
||||
final int localPort = serverSocket.getLocalPort();
|
||||
listener = new Listener(serverSocket, runner);
|
||||
final Thread listenThread = new Thread(listener);
|
||||
listenThread.setName("Listen to NiFi");
|
||||
listenThread.start();
|
||||
return localPort;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
final Listener listener = this.listener;
|
||||
if (listener == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
listener.stop();
|
||||
}
|
||||
|
||||
private class Listener implements Runnable {
|
||||
|
||||
private final ServerSocket serverSocket;
|
||||
private final ExecutorService executor;
|
||||
private final RunNiFi runner;
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
public Listener(final ServerSocket serverSocket, final RunNiFi runner) {
|
||||
this.serverSocket = serverSocket;
|
||||
this.executor = Executors.newFixedThreadPool(2);
|
||||
this.runner = runner;
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
stopped = true;
|
||||
|
||||
executor.shutdown();
|
||||
try {
|
||||
executor.awaitTermination(3, TimeUnit.SECONDS);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
|
||||
serverSocket.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!serverSocket.isClosed()) {
|
||||
try {
|
||||
if (stopped) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Socket socket;
|
||||
try {
|
||||
socket = serverSocket.accept();
|
||||
} catch (final IOException ioe) {
|
||||
if (stopped) {
|
||||
return;
|
||||
}
|
||||
|
||||
throw ioe;
|
||||
}
|
||||
|
||||
executor.submit(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
// we want to ensure that we don't try to read data from an InputStream directly
|
||||
// by a BufferedReader because any user on the system could open a socket and send
|
||||
// a multi-gigabyte file without any new lines in order to crash the Bootstrap,
|
||||
// which in turn may cause the Shutdown Hook to shutdown NiFi.
|
||||
// So we will limit the amount of data to read to 4 KB
|
||||
final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096);
|
||||
final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream());
|
||||
codec.communicate();
|
||||
} catch (final Throwable t) {
|
||||
System.out.println("Failed to communicate with NiFi due to " + t);
|
||||
t.printStackTrace();
|
||||
} finally {
|
||||
try {
|
||||
socket.close();
|
||||
} catch (final IOException ioe) {
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (final Throwable t) {
|
||||
System.err.println("Failed to receive information from NiFi due to " + t);
|
||||
t.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -24,63 +24,65 @@ import java.nio.charset.StandardCharsets;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class ShutdownHook extends Thread {
|
||||
private final Process nifiProcess;
|
||||
private final RunNiFi runner;
|
||||
private final int gracefulShutdownSeconds;
|
||||
|
||||
private volatile String secretKey;
|
||||
|
||||
public ShutdownHook(final Process nifiProcess, final RunNiFi runner, final String secretKey, final int gracefulShutdownSeconds) {
|
||||
this.nifiProcess = nifiProcess;
|
||||
this.runner = runner;
|
||||
this.secretKey = secretKey;
|
||||
this.gracefulShutdownSeconds = gracefulShutdownSeconds;
|
||||
}
|
||||
|
||||
void setSecretKey(final String secretKey) {
|
||||
this.secretKey = secretKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
runner.setAutoRestartNiFi(false);
|
||||
final int ccPort = runner.getNiFiCommandControlPort();
|
||||
if ( ccPort > 0 ) {
|
||||
System.out.println("Initiating Shutdown of NiFi...");
|
||||
|
||||
try {
|
||||
final Socket socket = new Socket("localhost", ccPort);
|
||||
final OutputStream out = socket.getOutputStream();
|
||||
out.write(("SHUTDOWN " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
|
||||
out.flush();
|
||||
|
||||
socket.close();
|
||||
} catch (final IOException ioe) {
|
||||
System.out.println("Failed to Shutdown NiFi due to " + ioe);
|
||||
}
|
||||
}
|
||||
|
||||
System.out.println("Waiting for Apache NiFi to finish shutting down...");
|
||||
final long startWait = System.nanoTime();
|
||||
while ( RunNiFi.isAlive(nifiProcess) ) {
|
||||
final long waitNanos = System.nanoTime() - startWait;
|
||||
final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
|
||||
if ( waitSeconds >= gracefulShutdownSeconds && gracefulShutdownSeconds > 0 ) {
|
||||
if ( RunNiFi.isAlive(nifiProcess) ) {
|
||||
System.out.println("NiFi has not finished shutting down after " + gracefulShutdownSeconds + " seconds. Killing process.");
|
||||
nifiProcess.destroy();
|
||||
}
|
||||
break;
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(1000L);
|
||||
} catch (final InterruptedException ie) {}
|
||||
}
|
||||
}
|
||||
|
||||
final File statusFile = runner.getStatusFile();
|
||||
if ( !statusFile.delete() ) {
|
||||
System.err.println("Failed to delete status file " + statusFile.getAbsolutePath() + "; this file should be cleaned up manually");
|
||||
}
|
||||
}
|
||||
|
||||
private final Process nifiProcess;
|
||||
private final RunNiFi runner;
|
||||
private final int gracefulShutdownSeconds;
|
||||
|
||||
private volatile String secretKey;
|
||||
|
||||
public ShutdownHook(final Process nifiProcess, final RunNiFi runner, final String secretKey, final int gracefulShutdownSeconds) {
|
||||
this.nifiProcess = nifiProcess;
|
||||
this.runner = runner;
|
||||
this.secretKey = secretKey;
|
||||
this.gracefulShutdownSeconds = gracefulShutdownSeconds;
|
||||
}
|
||||
|
||||
void setSecretKey(final String secretKey) {
|
||||
this.secretKey = secretKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
runner.setAutoRestartNiFi(false);
|
||||
final int ccPort = runner.getNiFiCommandControlPort();
|
||||
if (ccPort > 0) {
|
||||
System.out.println("Initiating Shutdown of NiFi...");
|
||||
|
||||
try {
|
||||
final Socket socket = new Socket("localhost", ccPort);
|
||||
final OutputStream out = socket.getOutputStream();
|
||||
out.write(("SHUTDOWN " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
|
||||
out.flush();
|
||||
|
||||
socket.close();
|
||||
} catch (final IOException ioe) {
|
||||
System.out.println("Failed to Shutdown NiFi due to " + ioe);
|
||||
}
|
||||
}
|
||||
|
||||
System.out.println("Waiting for Apache NiFi to finish shutting down...");
|
||||
final long startWait = System.nanoTime();
|
||||
while (RunNiFi.isAlive(nifiProcess)) {
|
||||
final long waitNanos = System.nanoTime() - startWait;
|
||||
final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
|
||||
if (waitSeconds >= gracefulShutdownSeconds && gracefulShutdownSeconds > 0) {
|
||||
if (RunNiFi.isAlive(nifiProcess)) {
|
||||
System.out.println("NiFi has not finished shutting down after " + gracefulShutdownSeconds + " seconds. Killing process.");
|
||||
nifiProcess.destroy();
|
||||
}
|
||||
break;
|
||||
} else {
|
||||
try {
|
||||
Thread.sleep(1000L);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final File statusFile = runner.getStatusFile();
|
||||
if (!statusFile.delete()) {
|
||||
System.err.println("Failed to delete status file " + statusFile.getAbsolutePath() + "; this file should be cleaned up manually");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,21 +17,22 @@
|
|||
package org.apache.nifi.bootstrap.exception;
|
||||
|
||||
public class InvalidCommandException extends Exception {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public InvalidCommandException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public InvalidCommandException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public InvalidCommandException(final Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
|
||||
public InvalidCommandException(final String message, final Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public InvalidCommandException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public InvalidCommandException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public InvalidCommandException(final Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
|
||||
public InvalidCommandException(final String message, final Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -78,7 +78,7 @@ public class FlowFilePackagerV1 implements FlowFilePackager {
|
|||
entry.setMode(tarPermissions);
|
||||
entry.setSize(fileSize);
|
||||
tarOut.putArchiveEntry(entry);
|
||||
final byte[] buffer = new byte[512 << 10];//512KB
|
||||
final byte[] buffer = new byte[512 << 10];//512KB
|
||||
int bytesRead = 0;
|
||||
while ((bytesRead = inStream.read(buffer)) != -1) { //still more data to read
|
||||
if (bytesRead > 0) {
|
||||
|
|
|
@ -51,7 +51,7 @@ public class FlowFileUnpackagerV1 implements FlowFileUnpackager {
|
|||
final TarArchiveEntry contentEntry = tarIn.getNextTarEntry();
|
||||
|
||||
if (contentEntry != null && contentEntry.getName().equals(FlowFilePackagerV1.FILENAME_CONTENT)) {
|
||||
final byte[] buffer = new byte[512 << 10];//512KB
|
||||
final byte[] buffer = new byte[512 << 10];//512KB
|
||||
int bytesRead = 0;
|
||||
while ((bytesRead = tarIn.read(buffer)) != -1) { //still more data to read
|
||||
if (bytesRead > 0) {
|
||||
|
|
|
@ -14,109 +14,115 @@
|
|||
limitations under the License.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-commons</artifactId>
|
||||
<version>0.1.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-commons</artifactId>
|
||||
<version>0.1.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<artifactId>nifi-hl7-query-language</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.7</source>
|
||||
<target>1.7</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.antlr</groupId>
|
||||
<artifactId>antlr3-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>antlr</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>src/test/resources/hypoglycemia</exclude>
|
||||
<exclude>src/test/resources/hyperglycemia</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
<artifactId>nifi-hl7-query-language</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.7</source>
|
||||
<target>1.7</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.antlr</groupId>
|
||||
<artifactId>antlr3-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>antlr</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>src/test/resources/hypoglycemia</exclude>
|
||||
<exclude>src/test/resources/hyperglycemia</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-checkstyle-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>**/HL7QueryParser.java,**/HL7QueryLexer.java</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.antlr</groupId>
|
||||
<artifactId>antlr-runtime</artifactId>
|
||||
<version>3.5.2</version>
|
||||
</dependency>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.antlr</groupId>
|
||||
<artifactId>antlr-runtime</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- HAPI to parse v2 messages -->
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-base</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v21</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v22</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v23</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v231</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v24</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v25</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v251</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v26</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<!-- HAPI to parse v2 messages -->
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-base</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v21</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v22</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v23</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v231</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v24</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v25</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v251</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>ca.uhn.hapi</groupId>
|
||||
<artifactId>hapi-structures-v26</artifactId>
|
||||
<version>2.2</version>
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -0,0 +1,22 @@
|
|||
<?xml version="1.0"?>
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<!DOCTYPE suppressions PUBLIC
|
||||
"-//Puppy Crawl//DTD Suppressions 1.1//EN"
|
||||
"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
|
||||
|
||||
<suppressions>
|
||||
<suppress files="[/\\]target[/\\]" checks=".*"/>
|
||||
</suppressions>
|
|
@ -24,14 +24,14 @@ import org.apache.nifi.hl7.model.HL7Field;
|
|||
|
||||
public class EmptyField implements HL7Field {
|
||||
|
||||
@Override
|
||||
public String getValue() {
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public String getValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HL7Component> getComponents() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
@Override
|
||||
public List<HL7Component> getComponents() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -32,52 +32,53 @@ 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<HL7Component> components;
|
||||
|
||||
public HapiField(final Type type) {
|
||||
this.value = PipeParser.encode(type, EncodingCharacters.defaultInstance());
|
||||
|
||||
final List<HL7Component> 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<HL7Component> getComponents() {
|
||||
return components;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return value;
|
||||
}
|
||||
private final String value;
|
||||
private final List<HL7Component> components;
|
||||
|
||||
public HapiField(final Type type) {
|
||||
this.value = PipeParser.encode(type, EncodingCharacters.defaultInstance());
|
||||
|
||||
final List<HL7Component> 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<HL7Component> getComponents() {
|
||||
return components;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return value;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,63 +32,64 @@ import ca.uhn.hl7v2.model.Segment;
|
|||
import ca.uhn.hl7v2.model.Structure;
|
||||
|
||||
public class HapiMessage implements HL7Message {
|
||||
private final Message message;
|
||||
private final List<HL7Segment> allSegments;
|
||||
private final Map<String, List<HL7Segment>> 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<HL7Segment> 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<HL7Segment> 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<HL7Segment> getSegments() {
|
||||
return Collections.unmodifiableList(allSegments);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HL7Segment> getSegments(final String segmentType) {
|
||||
final List<HL7Segment> segments = segmentMap.get(segmentType);
|
||||
if ( segments == null ) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
return Collections.unmodifiableList(segments);
|
||||
}
|
||||
private final Message message;
|
||||
private final List<HL7Segment> allSegments;
|
||||
private final Map<String, List<HL7Segment>> segmentMap;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return message.toString();
|
||||
}
|
||||
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<HL7Segment> 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<HL7Segment> 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<HL7Segment> getSegments() {
|
||||
return Collections.unmodifiableList(allSegments);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HL7Segment> getSegments(final String segmentType) {
|
||||
final List<HL7Segment> segments = segmentMap.get(segmentType);
|
||||
if (segments == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
return Collections.unmodifiableList(segments);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return message.toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,42 +28,42 @@ import ca.uhn.hl7v2.model.Segment;
|
|||
import ca.uhn.hl7v2.model.Type;
|
||||
|
||||
public class HapiSegment implements HL7Segment {
|
||||
private final Segment segment;
|
||||
private final List<HL7Field> fields;
|
||||
|
||||
public HapiSegment(final Segment segment) throws HL7Exception {
|
||||
this.segment = segment;
|
||||
|
||||
final List<HL7Field> 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();
|
||||
}
|
||||
private final Segment segment;
|
||||
private final List<HL7Field> fields;
|
||||
|
||||
@Override
|
||||
public List<HL7Field> getFields() {
|
||||
return fields;
|
||||
}
|
||||
public HapiSegment(final Segment segment) throws HL7Exception {
|
||||
this.segment = segment;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return segment.toString();
|
||||
}
|
||||
final List<HL7Field> 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<HL7Field> getFields() {
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return segment.toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,20 +23,21 @@ 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<HL7Component> getComponents() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
private final String value;
|
||||
|
||||
public SingleValueField(final String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HL7Component> getComponents() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,6 +22,6 @@ import org.apache.nifi.hl7.model.HL7Message;
|
|||
|
||||
public interface HL7Reader {
|
||||
|
||||
HL7Message nextMessage() throws IOException;
|
||||
|
||||
HL7Message nextMessage() throws IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -19,22 +19,23 @@ 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();
|
||||
}
|
||||
private static final long serialVersionUID = -5675416667224562441L;
|
||||
|
||||
public InvalidHL7Exception(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
public InvalidHL7Exception() {
|
||||
super();
|
||||
}
|
||||
|
||||
public InvalidHL7Exception(String message) {
|
||||
super(message);
|
||||
}
|
||||
public InvalidHL7Exception(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public InvalidHL7Exception(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public InvalidHL7Exception(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
|
||||
public InvalidHL7Exception(Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.nifi.hl7.model;
|
|||
import java.util.List;
|
||||
|
||||
public interface HL7Component {
|
||||
String getValue();
|
||||
List<HL7Component> getComponents();
|
||||
|
||||
String getValue();
|
||||
|
||||
List<HL7Component> getComponents();
|
||||
}
|
||||
|
|
|
@ -16,6 +16,5 @@
|
|||
*/
|
||||
package org.apache.nifi.hl7.model;
|
||||
|
||||
|
||||
public interface HL7Field extends HL7Component {
|
||||
}
|
||||
|
|
|
@ -20,8 +20,8 @@ import java.util.List;
|
|||
|
||||
public interface HL7Message {
|
||||
|
||||
List<HL7Segment> getSegments();
|
||||
|
||||
List<HL7Segment> getSegments(String segmentType);
|
||||
|
||||
List<HL7Segment> getSegments();
|
||||
|
||||
List<HL7Segment> getSegments(String segmentType);
|
||||
|
||||
}
|
||||
|
|
|
@ -20,8 +20,8 @@ import java.util.List;
|
|||
|
||||
public interface HL7Segment {
|
||||
|
||||
String getName();
|
||||
|
||||
List<HL7Field> getFields();
|
||||
|
||||
String getName();
|
||||
|
||||
List<HL7Field> getFields();
|
||||
|
||||
}
|
||||
|
|
|
@ -20,10 +20,10 @@ import org.apache.nifi.hl7.model.HL7Message;
|
|||
|
||||
public interface Declaration {
|
||||
|
||||
String getAlias();
|
||||
|
||||
boolean isRequired();
|
||||
|
||||
Object getDeclaredValue(HL7Message message);
|
||||
|
||||
String getAlias();
|
||||
|
||||
boolean isRequired();
|
||||
|
||||
Object getDeclaredValue(HL7Message message);
|
||||
|
||||
}
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
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;
|
||||
|
@ -58,196 +56,211 @@ import org.apache.nifi.hl7.query.result.StandardQueryResult;
|
|||
|
||||
import org.apache.nifi.hl7.query.antlr.HL7QueryLexer;
|
||||
import org.apache.nifi.hl7.query.antlr.HL7QueryParser;
|
||||
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.AND;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.DECLARE;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.DOT;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.EQUALS;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.GE;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.GT;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.IDENTIFIER;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.IS_NULL;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.LE;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.LT;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.MESSAGE;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.NOT;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.NOT_EQUALS;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.NOT_NULL;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.NUMBER;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.OR;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.REQUIRED;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.SEGMENT_NAME;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.SELECT;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.STRING_LITERAL;
|
||||
import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.WHERE;
|
||||
|
||||
public class HL7Query {
|
||||
private final Tree tree;
|
||||
private final String query;
|
||||
private final Set<Declaration> declarations = new HashSet<>();
|
||||
|
||||
private final List<Selection> selections;
|
||||
private final BooleanEvaluator whereEvaluator;
|
||||
|
||||
private HL7Query(final Tree tree, final String query) {
|
||||
this.tree = tree;
|
||||
this.query = query;
|
||||
|
||||
List<Selection> 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;
|
||||
}
|
||||
private final Tree tree;
|
||||
private final String query;
|
||||
private final Set<Declaration> declarations = new HashSet<>();
|
||||
|
||||
@Override
|
||||
public boolean isRequired() {
|
||||
return required;
|
||||
}
|
||||
private final List<Selection> selections;
|
||||
private final BooleanEvaluator whereEvaluator;
|
||||
|
||||
@Override
|
||||
public Object getDeclaredValue(final HL7Message message) {
|
||||
if ( message == null ) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return message.getSegments(segmentName);
|
||||
}
|
||||
};
|
||||
|
||||
declarations.add(declaration);
|
||||
}
|
||||
}
|
||||
private HL7Query(final Tree tree, final String query) {
|
||||
this.tree = tree;
|
||||
this.query = query;
|
||||
|
||||
private List<Selection> processSelect(final Tree select) {
|
||||
final List<Selection> selections = new ArrayList<>();
|
||||
List<Selection> select = null;
|
||||
BooleanEvaluator where = null;
|
||||
for (int i = 0; i < tree.getChildCount(); i++) {
|
||||
final Tree child = tree.getChild(i);
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
private BooleanEvaluator processWhere(final Tree where) {
|
||||
return buildBooleanEvaluator(where.getChild(0));
|
||||
}
|
||||
|
||||
this.whereEvaluator = where;
|
||||
this.selections = select;
|
||||
}
|
||||
|
||||
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 {
|
||||
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<Selection> processSelect(final Tree select) {
|
||||
final List<Selection> 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();
|
||||
|
@ -258,155 +271,114 @@ public class HL7Query {
|
|||
} catch (final Exception e) {
|
||||
throw new HL7QueryParsingException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static CommonTokenStream createTokenStream(final String expression) throws HL7QueryParsingException {
|
||||
}
|
||||
|
||||
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<Class<?>> getReturnTypes() {
|
||||
final List<Class<?>> 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<String, List<Object>> 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<Object> possibleValues;
|
||||
if ( value instanceof List ) {
|
||||
possibleValues = (List<Object>) value;
|
||||
} else if ( value instanceof Collection ) {
|
||||
possibleValues = new ArrayList<Object>((Collection<Object>) value);
|
||||
} else {
|
||||
possibleValues = new ArrayList<>(1);
|
||||
possibleValues.add(value);
|
||||
}
|
||||
|
||||
if ( possibleValues.isEmpty() ) {
|
||||
return new MissedResult(selections);
|
||||
}
|
||||
|
||||
possibleValueMap.put(declaration.getAlias(), possibleValues);
|
||||
totalIterations *= possibleValues.size();
|
||||
}
|
||||
public List<Class<?>> getReturnTypes() {
|
||||
final List<Class<?>> returnTypes = new ArrayList<>();
|
||||
|
||||
final Set<Map<String, Object>> resultSet = new HashSet<>();
|
||||
for (int i=0; i < totalIterations; i++) {
|
||||
final Map<String, Object> aliasValues = assignAliases(possibleValueMap, i);
|
||||
for (final Selection selection : selections) {
|
||||
returnTypes.add(selection.getEvaluator().getType());
|
||||
}
|
||||
|
||||
aliasValues.put(Evaluator.MESSAGE_KEY, message);
|
||||
if (whereEvaluator == null || Boolean.TRUE.equals(whereEvaluator.evaluate(aliasValues))) {
|
||||
final Map<String, Object> resultMap = new HashMap<>();
|
||||
return returnTypes;
|
||||
}
|
||||
|
||||
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<String, Object> assignAliases(final LinkedHashMap<String, List<Object>> possibleValues, final int iteration) {
|
||||
// final Map<String, Object> aliasMap = new HashMap<>();
|
||||
//
|
||||
// int aliasIndex = possibleValues.size() - 1;
|
||||
// for ( final Map.Entry<String, List<Object>> entry : possibleValues.entrySet() ) {
|
||||
// final String alias = entry.getKey();
|
||||
// final List<Object> 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<String, Object> assignAliases(final LinkedHashMap<String, List<Object>> possibleValues, final int iteration) {
|
||||
final Map<String, Object> aliasMap = new HashMap<>();
|
||||
|
||||
int divisor = 1;
|
||||
for ( final Map.Entry<String, List<Object>> entry : possibleValues.entrySet() ) {
|
||||
final String alias = entry.getKey();
|
||||
final List<Object> validValues = entry.getValue();
|
||||
@SuppressWarnings("unchecked")
|
||||
public QueryResult evaluate(final HL7Message message) {
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
int totalIterations = 1;
|
||||
final LinkedHashMap<String, List<Object>> 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<Object> possibleValues;
|
||||
if (value instanceof List) {
|
||||
possibleValues = (List<Object>) value;
|
||||
} else if (value instanceof Collection) {
|
||||
possibleValues = new ArrayList<Object>((Collection<Object>) 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<Map<String, Object>> resultSet = new HashSet<>();
|
||||
for (int i = 0; i < totalIterations; i++) {
|
||||
final Map<String, Object> aliasValues = assignAliases(possibleValueMap, i);
|
||||
|
||||
aliasValues.put(Evaluator.MESSAGE_KEY, message);
|
||||
if (whereEvaluator == null || Boolean.TRUE.equals(whereEvaluator.evaluate(aliasValues))) {
|
||||
final Map<String, Object> resultMap = new HashMap<>();
|
||||
|
||||
for (final Selection selection : selections) {
|
||||
final Object value = selection.getEvaluator().evaluate(aliasValues);
|
||||
resultMap.put(selection.getName(), value);
|
||||
}
|
||||
|
||||
resultSet.add(resultMap);
|
||||
}
|
||||
}
|
||||
|
||||
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<String, Object> assignAliases(final LinkedHashMap<String, List<Object>> possibleValues, final int iteration) {
|
||||
final Map<String, Object> aliasMap = new HashMap<>();
|
||||
|
||||
int divisor = 1;
|
||||
for (final Map.Entry<String, List<Object>> entry : possibleValues.entrySet()) {
|
||||
final String alias = entry.getKey();
|
||||
final List<Object> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,11 +19,12 @@ package org.apache.nifi.hl7.query;
|
|||
import java.util.List;
|
||||
|
||||
public interface QueryResult {
|
||||
boolean isMatch();
|
||||
|
||||
List<String> getLabels();
|
||||
|
||||
int getHitCount();
|
||||
|
||||
ResultHit nextHit();
|
||||
|
||||
boolean isMatch();
|
||||
|
||||
List<String> getLabels();
|
||||
|
||||
int getHitCount();
|
||||
|
||||
ResultHit nextHit();
|
||||
}
|
||||
|
|
|
@ -19,7 +19,8 @@ package org.apache.nifi.hl7.query;
|
|||
import java.util.Map;
|
||||
|
||||
public interface ResultHit {
|
||||
Object getValue(String label);
|
||||
|
||||
Map<String, Object> getSelectedValues();
|
||||
|
||||
Object getValue(String label);
|
||||
|
||||
Map<String, Object> getSelectedValues();
|
||||
}
|
||||
|
|
|
@ -19,19 +19,20 @@ 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;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.nifi.hl7.query.evaluator;
|
|||
|
||||
public abstract class BooleanEvaluator implements Evaluator<Boolean> {
|
||||
|
||||
public Class<? extends Boolean> getType() {
|
||||
return Boolean.class;
|
||||
}
|
||||
public Class<? extends Boolean> getType() {
|
||||
return Boolean.class;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,10 @@ package org.apache.nifi.hl7.query.evaluator;
|
|||
import java.util.Map;
|
||||
|
||||
public interface Evaluator<T> {
|
||||
public static final String MESSAGE_KEY = "message";
|
||||
|
||||
T evaluate(Map<String, Object> objectMap);
|
||||
|
||||
Class<? extends T> getType();
|
||||
|
||||
public static final String MESSAGE_KEY = "message";
|
||||
|
||||
T evaluate(Map<String, Object> objectMap);
|
||||
|
||||
Class<? extends T> getType();
|
||||
}
|
||||
|
|
|
@ -16,11 +16,10 @@
|
|||
*/
|
||||
package org.apache.nifi.hl7.query.evaluator;
|
||||
|
||||
|
||||
public abstract class IntegerEvaluator implements Evaluator<Integer> {
|
||||
|
||||
public Class<? extends Integer> getType() {
|
||||
return Integer.class;
|
||||
}
|
||||
public Class<? extends Integer> getType() {
|
||||
return Integer.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -18,8 +18,8 @@ package org.apache.nifi.hl7.query.evaluator;
|
|||
|
||||
public abstract class StringEvaluator implements Evaluator<String> {
|
||||
|
||||
public Class<? extends String> getType() {
|
||||
return String.class;
|
||||
}
|
||||
|
||||
public Class<? extends String> getType() {
|
||||
return String.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,83 +24,88 @@ 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<String, Object> 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();
|
||||
}
|
||||
private final Evaluator<?> lhs;
|
||||
private final Evaluator<?> rhs;
|
||||
|
||||
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);
|
||||
public AbstractComparisonEvaluator(final Evaluator<?> lhs, final Evaluator<?> rhs) {
|
||||
this.lhs = lhs;
|
||||
this.rhs = rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Boolean evaluate(final Map<String, Object> 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);
|
||||
}
|
||||
|
|
|
@ -21,47 +21,48 @@ 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);
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -20,13 +20,13 @@ 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()));
|
||||
}
|
||||
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()));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -18,17 +18,15 @@ 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);
|
||||
}
|
||||
public GreaterThanEvaluator(final Evaluator<?> lhs, final Evaluator<?> rhs) {
|
||||
super(lhs, rhs);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean compareNumbers(final Double lhs, final Double rhs) {
|
||||
return lhs > rhs;
|
||||
}
|
||||
@Override
|
||||
protected boolean compareNumbers(final Double lhs, final Double rhs) {
|
||||
return lhs > rhs;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -18,17 +18,15 @@ 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);
|
||||
}
|
||||
public GreaterThanOrEqualEvaluator(final Evaluator<?> lhs, final Evaluator<?> rhs) {
|
||||
super(lhs, rhs);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean compareNumbers(final Double lhs, final Double rhs) {
|
||||
return lhs >= rhs;
|
||||
}
|
||||
@Override
|
||||
protected boolean compareNumbers(final Double lhs, final Double rhs) {
|
||||
return lhs >= rhs;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -24,46 +24,47 @@ 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<String, Object> 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;
|
||||
}
|
||||
private final Evaluator<?> subjectEvaluator;
|
||||
|
||||
public IsNullEvaluator(final Evaluator<?> subjectEvaluator) {
|
||||
this.subjectEvaluator = subjectEvaluator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean evaluate(final Map<String, Object> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,13 +19,14 @@ 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;
|
||||
}
|
||||
public LessThanEvaluator(final Evaluator<?> lhs, final Evaluator<?> rhs) {
|
||||
super(lhs, rhs);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean compareNumbers(final Double lhs, final Double rhs) {
|
||||
return lhs < rhs;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,13 +19,14 @@ 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;
|
||||
}
|
||||
public LessThanOrEqualEvaluator(final Evaluator<?> lhs, final Evaluator<?> rhs) {
|
||||
super(lhs, rhs);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean compareNumbers(final Double lhs, final Double rhs) {
|
||||
return lhs <= rhs;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,14 +19,14 @@ 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());
|
||||
}
|
||||
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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,16 +21,17 @@ 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<String, Object> objectMap) {
|
||||
final Boolean subjectValue = subjectEvaluator.evaluate(objectMap);
|
||||
return (subjectValue == null || Boolean.TRUE.equals(subjectValue));
|
||||
}
|
||||
|
||||
private final BooleanEvaluator subjectEvaluator;
|
||||
|
||||
public NotEvaluator(final BooleanEvaluator subjectEvaluator) {
|
||||
this.subjectEvaluator = subjectEvaluator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean evaluate(final Map<String, Object> objectMap) {
|
||||
final Boolean subjectValue = subjectEvaluator.evaluate(objectMap);
|
||||
return (subjectValue == null || Boolean.TRUE.equals(subjectValue));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,42 +24,43 @@ 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<String, Object> 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;
|
||||
}
|
||||
private final Evaluator<?> subjectEvaluator;
|
||||
|
||||
public NotNullEvaluator(final Evaluator<?> subjectEvaluator) {
|
||||
this.subjectEvaluator = subjectEvaluator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean evaluate(final Map<String, Object> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,16 +21,16 @@ 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<String, Object> objectMap) {
|
||||
return value;
|
||||
}
|
||||
|
||||
private final Integer value;
|
||||
|
||||
public IntegerLiteralEvaluator(final Integer value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer evaluate(final Map<String, Object> objectMap) {
|
||||
return value;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,15 +21,16 @@ 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<String, Object> objectMap) {
|
||||
return value;
|
||||
}
|
||||
|
||||
private final String value;
|
||||
|
||||
public StringLiteralEvaluator(final String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String evaluate(final Map<String, Object> objectMap) {
|
||||
return value;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,23 +21,24 @@ 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<String, Object> 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));
|
||||
}
|
||||
|
||||
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<String, Object> 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));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,23 +21,24 @@ 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<String, Object> 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));
|
||||
}
|
||||
|
||||
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<String, Object> 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));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,21 +22,22 @@ import org.apache.nifi.hl7.query.evaluator.Evaluator;
|
|||
import org.apache.nifi.hl7.query.evaluator.StringEvaluator;
|
||||
|
||||
public class DeclaredReferenceEvaluator implements Evaluator<Object> {
|
||||
private final StringEvaluator referenceNameEvaluator;
|
||||
|
||||
public DeclaredReferenceEvaluator(final StringEvaluator referenceNameEvaluator) {
|
||||
this.referenceNameEvaluator = referenceNameEvaluator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object evaluate(final Map<String, Object> objectMap) {
|
||||
final String referenceName = referenceNameEvaluator.evaluate(objectMap);
|
||||
return objectMap.get(referenceName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<? extends Object> getType() {
|
||||
return Object.class;
|
||||
}
|
||||
private final StringEvaluator referenceNameEvaluator;
|
||||
|
||||
public DeclaredReferenceEvaluator(final StringEvaluator referenceNameEvaluator) {
|
||||
this.referenceNameEvaluator = referenceNameEvaluator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object evaluate(final Map<String, Object> objectMap) {
|
||||
final String referenceName = referenceNameEvaluator.evaluate(objectMap);
|
||||
return objectMap.get(referenceName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<? extends Object> getType() {
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -28,61 +28,62 @@ import org.apache.nifi.hl7.query.evaluator.Evaluator;
|
|||
import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator;
|
||||
|
||||
public class DotEvaluator implements Evaluator<Object> {
|
||||
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<String, Object> objectMap) {
|
||||
final Object lhsValue = this.lhs.evaluate(objectMap);
|
||||
final Integer rhsValue = this.rhs.evaluate(objectMap);
|
||||
|
||||
if ( lhsValue == null || rhsValue == null ) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final List<Object> 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<? extends Object> getType() {
|
||||
return Object.class;
|
||||
}
|
||||
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<String, Object> objectMap) {
|
||||
final Object lhsValue = this.lhs.evaluate(objectMap);
|
||||
final Integer rhsValue = this.rhs.evaluate(objectMap);
|
||||
|
||||
if (lhsValue == null || rhsValue == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final List<Object> 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<? extends Object> getType() {
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -28,40 +28,41 @@ import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator;
|
|||
|
||||
@SuppressWarnings("rawtypes")
|
||||
public class FieldEvaluator implements Evaluator<List> {
|
||||
private final SegmentEvaluator segmentEvaluator;
|
||||
private final IntegerEvaluator indexEvaluator;
|
||||
|
||||
public FieldEvaluator(final SegmentEvaluator segmentEvaluator, final IntegerEvaluator indexEvaluator) {
|
||||
this.segmentEvaluator = segmentEvaluator;
|
||||
this.indexEvaluator = indexEvaluator;
|
||||
}
|
||||
|
||||
public List<HL7Field> evaluate(final Map<String, Object> objectMap) {
|
||||
final List<HL7Segment> segments = segmentEvaluator.evaluate(objectMap);
|
||||
if ( segments == null ) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final Integer index = indexEvaluator.evaluate(objectMap);
|
||||
if ( index == null ) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final List<HL7Field> fields = new ArrayList<>();
|
||||
for ( final HL7Segment segment : segments ) {
|
||||
final List<HL7Field> segmentFields = segment.getFields();
|
||||
if ( segmentFields.size() <= index ) {
|
||||
continue;
|
||||
}
|
||||
|
||||
fields.add(segmentFields.get(index));
|
||||
}
|
||||
|
||||
return fields;
|
||||
}
|
||||
|
||||
public Class<? extends List> getType() {
|
||||
return List.class;
|
||||
}
|
||||
private final SegmentEvaluator segmentEvaluator;
|
||||
private final IntegerEvaluator indexEvaluator;
|
||||
|
||||
public FieldEvaluator(final SegmentEvaluator segmentEvaluator, final IntegerEvaluator indexEvaluator) {
|
||||
this.segmentEvaluator = segmentEvaluator;
|
||||
this.indexEvaluator = indexEvaluator;
|
||||
}
|
||||
|
||||
public List<HL7Field> evaluate(final Map<String, Object> objectMap) {
|
||||
final List<HL7Segment> segments = segmentEvaluator.evaluate(objectMap);
|
||||
if (segments == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final Integer index = indexEvaluator.evaluate(objectMap);
|
||||
if (index == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final List<HL7Field> fields = new ArrayList<>();
|
||||
for (final HL7Segment segment : segments) {
|
||||
final List<HL7Field> segmentFields = segment.getFields();
|
||||
if (segmentFields.size() <= index) {
|
||||
continue;
|
||||
}
|
||||
|
||||
fields.add(segmentFields.get(index));
|
||||
}
|
||||
|
||||
return fields;
|
||||
}
|
||||
|
||||
public Class<? extends List> getType() {
|
||||
return List.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -23,12 +23,12 @@ import org.apache.nifi.hl7.query.evaluator.Evaluator;
|
|||
|
||||
public class MessageEvaluator implements Evaluator<HL7Message> {
|
||||
|
||||
public HL7Message evaluate(final Map<String, Object> objectMap) {
|
||||
return (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY);
|
||||
}
|
||||
public HL7Message evaluate(final Map<String, Object> objectMap) {
|
||||
return (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY);
|
||||
}
|
||||
|
||||
public Class<? extends HL7Message> getType() {
|
||||
return HL7Message.class;
|
||||
}
|
||||
public Class<? extends HL7Message> getType() {
|
||||
return HL7Message.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,25 +27,26 @@ import org.apache.nifi.hl7.query.evaluator.StringEvaluator;
|
|||
|
||||
@SuppressWarnings("rawtypes")
|
||||
public class SegmentEvaluator implements Evaluator<List> {
|
||||
private final StringEvaluator segmentTypeEvaluator;
|
||||
|
||||
public SegmentEvaluator(final StringEvaluator segmentTypeEvaluator) {
|
||||
this.segmentTypeEvaluator = segmentTypeEvaluator;
|
||||
}
|
||||
|
||||
public List<HL7Segment> evaluate(final Map<String, Object> objectMap) {
|
||||
final String segmentType = segmentTypeEvaluator.evaluate(objectMap);
|
||||
if ( segmentType == null ) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final HL7Message message = (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY);
|
||||
final List<HL7Segment> segments = message.getSegments(segmentType);
|
||||
return (segments == null) ? Collections.<HL7Segment>emptyList() : segments;
|
||||
}
|
||||
|
||||
public Class<? extends List> getType() {
|
||||
return List.class;
|
||||
}
|
||||
private final StringEvaluator segmentTypeEvaluator;
|
||||
|
||||
public SegmentEvaluator(final StringEvaluator segmentTypeEvaluator) {
|
||||
this.segmentTypeEvaluator = segmentTypeEvaluator;
|
||||
}
|
||||
|
||||
public List<HL7Segment> evaluate(final Map<String, Object> objectMap) {
|
||||
final String segmentType = segmentTypeEvaluator.evaluate(objectMap);
|
||||
if (segmentType == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final HL7Message message = (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY);
|
||||
final List<HL7Segment> segments = message.getSegments(segmentType);
|
||||
return (segments == null) ? Collections.<HL7Segment>emptyList() : segments;
|
||||
}
|
||||
|
||||
public Class<? extends List> getType() {
|
||||
return List.class;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,21 +17,22 @@
|
|||
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);
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,33 +24,34 @@ import org.apache.nifi.hl7.query.ResultHit;
|
|||
import org.apache.nifi.hl7.query.Selection;
|
||||
|
||||
public class MissedResult implements QueryResult {
|
||||
private final List<Selection> selections;
|
||||
|
||||
public MissedResult(final List<Selection> selections) {
|
||||
this.selections = selections;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getLabels() {
|
||||
final List<String> labels = new ArrayList<>();
|
||||
for ( final Selection selection : selections ) {
|
||||
labels.add(selection.getName());
|
||||
}
|
||||
return labels;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMatch() {
|
||||
return false;
|
||||
}
|
||||
private final List<Selection> selections;
|
||||
|
||||
@Override
|
||||
public ResultHit nextHit() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getHitCount() {
|
||||
return 0;
|
||||
}
|
||||
public MissedResult(final List<Selection> selections) {
|
||||
this.selections = selections;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getLabels() {
|
||||
final List<String> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,43 +27,44 @@ import org.apache.nifi.hl7.query.ResultHit;
|
|||
import org.apache.nifi.hl7.query.Selection;
|
||||
|
||||
public class StandardQueryResult implements QueryResult {
|
||||
private final List<Selection> selections;
|
||||
private final Set<Map<String, Object>> hits;
|
||||
private final Iterator<Map<String, Object>> hitIterator;
|
||||
|
||||
public StandardQueryResult(final List<Selection> selections, final Set<Map<String, Object>> hits) {
|
||||
this.selections = selections;
|
||||
this.hits = hits;
|
||||
|
||||
hitIterator = hits.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMatch() {
|
||||
return !hits.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getLabels() {
|
||||
final List<String> labels = new ArrayList<>();
|
||||
for ( final Selection selection : selections ) {
|
||||
labels.add(selection.getName());
|
||||
}
|
||||
return labels;
|
||||
}
|
||||
private final List<Selection> selections;
|
||||
private final Set<Map<String, Object>> hits;
|
||||
private final Iterator<Map<String, Object>> hitIterator;
|
||||
|
||||
public StandardQueryResult(final List<Selection> selections, final Set<Map<String, Object>> hits) {
|
||||
this.selections = selections;
|
||||
this.hits = hits;
|
||||
|
||||
hitIterator = hits.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMatch() {
|
||||
return !hits.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getLabels() {
|
||||
final List<String> 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;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getHitCount() {
|
||||
return hits.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultHit nextHit() {
|
||||
if ( hitIterator.hasNext() ) {
|
||||
return new StandardResultHit(hitIterator.next());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,20 +22,21 @@ import java.util.Map;
|
|||
import org.apache.nifi.hl7.query.ResultHit;
|
||||
|
||||
public class StandardResultHit implements ResultHit {
|
||||
private final Map<String, Object> values;
|
||||
|
||||
public StandardResultHit(final Map<String, Object> values) {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getValue(final String label) {
|
||||
return values.get(label);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getSelectedValues() {
|
||||
return Collections.unmodifiableMap(values);
|
||||
}
|
||||
private final Map<String, Object> values;
|
||||
|
||||
public StandardResultHit(final Map<String, Object> values) {
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getValue(final String label) {
|
||||
return values.get(label);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getSelectedValues() {
|
||||
return Collections.unmodifiableMap(values);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -43,268 +43,264 @@ import ca.uhn.hl7v2.validation.impl.ValidationContextFactory;
|
|||
@SuppressWarnings("resource")
|
||||
public class TestHL7Query {
|
||||
|
||||
@Test
|
||||
public void testAssignAliases() {
|
||||
final LinkedHashMap<String, List<Object>> possibleValueMap = new LinkedHashMap<>();
|
||||
|
||||
final List<Object> valuesA = new ArrayList<>();
|
||||
valuesA.add("a");
|
||||
valuesA.add("b");
|
||||
valuesA.add("c");
|
||||
|
||||
final List<Object> valuesB = new ArrayList<>();
|
||||
valuesB.add("d");
|
||||
|
||||
final List<Object> valuesC = new ArrayList<>();
|
||||
valuesC.add("e");
|
||||
valuesC.add("f");
|
||||
|
||||
final List<Object> 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<String, Object> 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/hypoglycemia"));
|
||||
final QueryResult result = query.evaluate(msg);
|
||||
assertTrue(result.isMatch());
|
||||
final List<String> 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/hypoglycemia"));
|
||||
final QueryResult result = query.evaluate(msg);
|
||||
assertTrue(result.isMatch());
|
||||
final List<String> 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<Object> nameList = (List) names;
|
||||
assertEquals(1, nameList.size());
|
||||
final HL7Field nameField = (HL7Field) nameList.get(0);
|
||||
assertEquals("SMITH^JOHN", 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/hypoglycemia")));
|
||||
assertTrue( 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() );
|
||||
@Test
|
||||
public void testAssignAliases() {
|
||||
final LinkedHashMap<String, List<Object>> possibleValueMap = new LinkedHashMap<>();
|
||||
|
||||
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() );
|
||||
final List<Object> valuesA = new ArrayList<>();
|
||||
valuesA.add("a");
|
||||
valuesA.add("b");
|
||||
valuesA.add("c");
|
||||
|
||||
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() );
|
||||
final List<Object> valuesB = new ArrayList<>();
|
||||
valuesB.add("d");
|
||||
|
||||
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() );
|
||||
final List<Object> valuesC = new ArrayList<>();
|
||||
valuesC.add("e");
|
||||
valuesC.add("f");
|
||||
|
||||
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() );
|
||||
final List<Object> valuesD = new ArrayList<>();
|
||||
valuesD.add("g");
|
||||
valuesD.add("h");
|
||||
|
||||
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() );
|
||||
possibleValueMap.put("A", valuesA);
|
||||
possibleValueMap.put("B", valuesB);
|
||||
possibleValueMap.put("C", valuesC);
|
||||
possibleValueMap.put("D", valuesD);
|
||||
|
||||
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() );
|
||||
}
|
||||
for (int i = 0; i < valuesA.size() * valuesB.size() * valuesC.size() * valuesD.size(); i++) {
|
||||
System.out.println(i + " : " + HL7Query.assignAliases(possibleValueMap, i));
|
||||
}
|
||||
|
||||
|
||||
@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());
|
||||
}
|
||||
|
||||
@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() );
|
||||
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");
|
||||
}
|
||||
|
||||
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() );
|
||||
private void verifyAssignments(final Map<String, Object> 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"));
|
||||
}
|
||||
|
||||
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 testSelectMessage() throws HL7Exception, IOException {
|
||||
final HL7Query query = HL7Query.compile("SELECT MESSAGE");
|
||||
final HL7Message msg = createMessage(new File("src/test/resources/hypoglycemia"));
|
||||
final QueryResult result = query.evaluate(msg);
|
||||
assertTrue(result.isMatch());
|
||||
final List<String> labels = result.getLabels();
|
||||
assertEquals(1, labels.size());
|
||||
assertEquals("MESSAGE", labels.get(0));
|
||||
|
||||
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() );
|
||||
assertEquals(1, result.getHitCount());
|
||||
assertEquals(msg, result.nextHit().getValue("MESSAGE"));
|
||||
}
|
||||
|
||||
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
|
||||
@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/hypoglycemia"));
|
||||
final QueryResult result = query.evaluate(msg);
|
||||
assertTrue(result.isMatch());
|
||||
final List<String> 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<Object> nameList = (List) names;
|
||||
assertEquals(1, nameList.size());
|
||||
final HL7Field nameField = (HL7Field) nameList.get(0);
|
||||
assertEquals("SMITH^JOHN", 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/hypoglycemia")));
|
||||
assertTrue(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());
|
||||
}
|
||||
|
||||
@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());
|
||||
}
|
||||
|
||||
@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());
|
||||
}
|
||||
|
||||
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
|
||||
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() );
|
||||
}
|
||||
|
||||
|
||||
@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() );
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -360,7 +360,7 @@ public class NiFiProperties extends Properties {
|
|||
/**
|
||||
* The socket port to listen on for a Remote Input Port.
|
||||
*
|
||||
* @return
|
||||
* @return the remote input port
|
||||
*/
|
||||
public Integer getRemoteInputPort() {
|
||||
return getPropertyAsPort(REMOTE_INPUT_PORT, DEFAULT_REMOTE_INPUT_PORT);
|
||||
|
@ -383,7 +383,7 @@ public class NiFiProperties extends Properties {
|
|||
/**
|
||||
* Returns the directory to which Templates are to be persisted
|
||||
*
|
||||
* @return
|
||||
* @return the template directory
|
||||
*/
|
||||
public Path getTemplateDirectory() {
|
||||
final String strVal = getProperty(TEMPLATE_DIRECTORY);
|
||||
|
@ -414,7 +414,7 @@ public class NiFiProperties extends Properties {
|
|||
* Returns the number of partitions that should be used for the FlowFile
|
||||
* Repository
|
||||
*
|
||||
* @return
|
||||
* @return the number of partitions
|
||||
*/
|
||||
public int getFlowFileRepositoryPartitions() {
|
||||
final String rawProperty = getProperty(FLOWFILE_REPOSITORY_PARTITIONS, DEFAULT_FLOWFILE_REPO_PARTITIONS);
|
||||
|
@ -425,7 +425,7 @@ public class NiFiProperties extends Properties {
|
|||
* Returns the number of milliseconds between FlowFileRepository
|
||||
* checkpointing
|
||||
*
|
||||
* @return
|
||||
* @return the number of milliseconds between checkpoint events
|
||||
*/
|
||||
public String getFlowFileRepositoryCheckpointInterval() {
|
||||
return getProperty(FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL, DEFAULT_FLOWFILE_CHECKPOINT_INTERVAL);
|
||||
|
@ -504,7 +504,7 @@ public class NiFiProperties extends Properties {
|
|||
public int getWebThreads() {
|
||||
return getIntegerProperty(WEB_THREADS, DEFAULT_WEB_THREADS);
|
||||
}
|
||||
|
||||
|
||||
public File getWebWorkingDirectory() {
|
||||
return new File(getProperty(WEB_WORKING_DIR, DEFAULT_WEB_WORKING_DIR));
|
||||
}
|
||||
|
@ -551,7 +551,7 @@ public class NiFiProperties extends Properties {
|
|||
/**
|
||||
* Returns the auto refresh interval in seconds.
|
||||
*
|
||||
* @return
|
||||
* @return the interval over which the properties should auto refresh
|
||||
*/
|
||||
public String getAutoRefreshInterval() {
|
||||
return getProperty(UI_AUTO_REFRESH_INTERVAL);
|
||||
|
@ -829,7 +829,7 @@ public class NiFiProperties extends Properties {
|
|||
* values configured. No directories will be created as a result of this
|
||||
* operation.
|
||||
*
|
||||
* @return
|
||||
* @return the name and paths of all provenance repository locations
|
||||
*/
|
||||
public Map<String, Path> getProvenanceRepositoryPaths() {
|
||||
final Map<String, Path> provenanceRepositoryPaths = new HashMap<>();
|
||||
|
|
|
@ -36,10 +36,11 @@ public final class CertificateUtils {
|
|||
/**
|
||||
* Returns true if the given keystore can be loaded using the given keystore
|
||||
* type and password. Returns false otherwise.
|
||||
* @param keystore
|
||||
* @param keystoreType
|
||||
* @param password
|
||||
* @return
|
||||
*
|
||||
* @param keystore the keystore to validate
|
||||
* @param keystoreType the type of the keystore
|
||||
* @param password the password to access the keystore
|
||||
* @return true if valid; false otherwise
|
||||
*/
|
||||
public static boolean isStoreValid(final URL keystore, final KeystoreType keystoreType, final char[] password) {
|
||||
|
||||
|
@ -81,8 +82,8 @@ public final class CertificateUtils {
|
|||
* returned. If the CN cannot be extracted because the DN is in an
|
||||
* unrecognized format, the entire DN is returned.
|
||||
*
|
||||
* @param dn
|
||||
* @return
|
||||
* @param dn the dn to extract the username from
|
||||
* @return the exatracted username
|
||||
*/
|
||||
public static String extractUsername(String dn) {
|
||||
String username = dn;
|
||||
|
@ -135,11 +136,11 @@ public final class CertificateUtils {
|
|||
|
||||
final List<String> result = new ArrayList<>();
|
||||
for (final List<?> generalName : altNames) {
|
||||
/*
|
||||
* generalName has the name type as the first element a String or
|
||||
* byte array for the second element. We return any general names
|
||||
/**
|
||||
* generalName has the name type as the first element a String or
|
||||
* byte array for the second element. We return any general names
|
||||
* that are String types.
|
||||
*
|
||||
*
|
||||
* We don't inspect the numeric name type because some certificates
|
||||
* incorrectly put IPs and DNS names under the wrong name types.
|
||||
*/
|
||||
|
|
|
@ -100,12 +100,12 @@ public enum SecurityStoreTypes {
|
|||
/**
|
||||
* Creates an instance.
|
||||
*
|
||||
* @param storeProperty the Java system property for setting the keystore (
|
||||
* or truststore) path
|
||||
* @param storeProperty the Java system property for setting the keystore or
|
||||
* truststore path
|
||||
* @param storePasswordProperty the Java system property for setting the
|
||||
* keystore (or truststore) password
|
||||
* keystore or truststore path
|
||||
* @param storeTypeProperty the Java system property for setting the
|
||||
* keystore (or truststore) type
|
||||
* keystore or truststore type
|
||||
*/
|
||||
SecurityStoreTypes(final String storeProperty,
|
||||
final String storePasswordProperty,
|
||||
|
|
|
@ -60,12 +60,12 @@ public final class SslContextFactory {
|
|||
* @param clientAuth the type of client authentication
|
||||
*
|
||||
* @return a SSLContext instance
|
||||
* @throws java.security.KeyStoreException
|
||||
* @throws java.io.IOException
|
||||
* @throws java.security.NoSuchAlgorithmException
|
||||
* @throws java.security.cert.CertificateException
|
||||
* @throws java.security.UnrecoverableKeyException
|
||||
* @throws java.security.KeyManagementException
|
||||
* @throws java.security.KeyStoreException if any issues accessing the keystore
|
||||
* @throws java.io.IOException for any problems loading the keystores
|
||||
* @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
|
||||
* @throws java.security.cert.CertificateException if there is an issue with the certificate
|
||||
* @throws java.security.UnrecoverableKeyException if the key is insufficient
|
||||
* @throws java.security.KeyManagementException if unable to manage the key
|
||||
*/
|
||||
public static SSLContext createSslContext(
|
||||
final String keystore, final char[] keystorePasswd, final String keystoreType,
|
||||
|
@ -113,12 +113,12 @@ public final class SslContextFactory {
|
|||
* @param keystoreType the type of keystore (e.g., PKCS12, JKS)
|
||||
*
|
||||
* @return a SSLContext instance
|
||||
* @throws java.security.KeyStoreException
|
||||
* @throws java.io.IOException
|
||||
* @throws java.security.NoSuchAlgorithmException
|
||||
* @throws java.security.cert.CertificateException
|
||||
* @throws java.security.UnrecoverableKeyException
|
||||
* @throws java.security.KeyManagementException
|
||||
* @throws java.security.KeyStoreException if any issues accessing the keystore
|
||||
* @throws java.io.IOException for any problems loading the keystores
|
||||
* @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
|
||||
* @throws java.security.cert.CertificateException if there is an issue with the certificate
|
||||
* @throws java.security.UnrecoverableKeyException if the key is insufficient
|
||||
* @throws java.security.KeyManagementException if unable to manage the key
|
||||
*/
|
||||
public static SSLContext createSslContext(
|
||||
final String keystore, final char[] keystorePasswd, final String keystoreType)
|
||||
|
@ -149,12 +149,12 @@ public final class SslContextFactory {
|
|||
* @param truststoreType the type of truststore (e.g., PKCS12, JKS)
|
||||
*
|
||||
* @return a SSLContext instance
|
||||
* @throws java.security.KeyStoreException
|
||||
* @throws java.io.IOException
|
||||
* @throws java.security.NoSuchAlgorithmException
|
||||
* @throws java.security.cert.CertificateException
|
||||
* @throws java.security.UnrecoverableKeyException
|
||||
* @throws java.security.KeyManagementException
|
||||
* @throws java.security.KeyStoreException if any issues accessing the keystore
|
||||
* @throws java.io.IOException for any problems loading the keystores
|
||||
* @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
|
||||
* @throws java.security.cert.CertificateException if there is an issue with the certificate
|
||||
* @throws java.security.UnrecoverableKeyException if the key is insufficient
|
||||
* @throws java.security.KeyManagementException if unable to manage the key
|
||||
*/
|
||||
public static SSLContext createTrustSslContext(
|
||||
final String truststore, final char[] truststorePasswd, final String truststoreType)
|
||||
|
|
|
@ -17,53 +17,50 @@
|
|||
package org.apache.nifi.flowfile.attributes;
|
||||
|
||||
public enum CoreAttributes implements FlowFileAttributeKey {
|
||||
|
||||
/**
|
||||
* The flowfile's path indicates the relative directory to which a FlowFile belongs and does not
|
||||
* contain the filename
|
||||
* The flowfile's path indicates the relative directory to which a FlowFile
|
||||
* belongs and does not contain the filename
|
||||
*/
|
||||
PATH("path"),
|
||||
|
||||
/**
|
||||
* The flowfile's absolute path indicates the absolute directory to which a FlowFile belongs and does not
|
||||
* contain the filename
|
||||
* The flowfile's absolute path indicates the absolute directory to which a
|
||||
* FlowFile belongs and does not contain the filename
|
||||
*/
|
||||
ABSOLUTE_PATH("absolute.path"),
|
||||
|
||||
/**
|
||||
* The filename of the FlowFile. The filename should not contain any directory structure.
|
||||
* The filename of the FlowFile. The filename should not contain any
|
||||
* directory structure.
|
||||
*/
|
||||
FILENAME("filename"),
|
||||
|
||||
/**
|
||||
* A unique UUID assigned to this FlowFile
|
||||
*/
|
||||
UUID("uuid"),
|
||||
|
||||
/**
|
||||
* A numeric value indicating the FlowFile priority
|
||||
*/
|
||||
PRIORITY("priority"),
|
||||
|
||||
/**
|
||||
* The MIME Type of this FlowFile
|
||||
*/
|
||||
MIME_TYPE("mime.type"),
|
||||
|
||||
/**
|
||||
* Specifies the reason that a FlowFile is being discarded
|
||||
*/
|
||||
DISCARD_REASON("discard.reason"),
|
||||
|
||||
/**
|
||||
* Indicates an identifier other than the FlowFile's UUID that is known to refer to this FlowFile.
|
||||
* Indicates an identifier other than the FlowFile's UUID that is known to
|
||||
* refer to this FlowFile.
|
||||
*/
|
||||
ALTERNATE_IDENTIFIER("alternate.identifier");
|
||||
|
||||
|
||||
private final String key;
|
||||
|
||||
private CoreAttributes(final String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String key() {
|
||||
return key;
|
||||
|
|
|
@ -17,5 +17,6 @@
|
|||
package org.apache.nifi.flowfile.attributes;
|
||||
|
||||
public interface FlowFileAttributeKey {
|
||||
|
||||
String key();
|
||||
}
|
||||
|
|
|
@ -29,7 +29,7 @@ public interface VersionNegotiator {
|
|||
* Sets the version of this resource to the specified version. Only the
|
||||
* lower byte of the version is relevant.
|
||||
*
|
||||
* @param version
|
||||
* @param version the version to set
|
||||
* @throws IllegalArgumentException if the given Version is not supported by
|
||||
* this resource, as is indicated by the {@link #isVersionSupported(int)}
|
||||
* method
|
||||
|
@ -47,8 +47,8 @@ public interface VersionNegotiator {
|
|||
* given maxVersion. If no acceptable version exists that is less than
|
||||
* <code>maxVersion</code>, then <code>null</code> is returned
|
||||
*
|
||||
* @param maxVersion
|
||||
* @return
|
||||
* @param maxVersion the maximum version desired
|
||||
* @return the preferred version if found; null otherwise
|
||||
*/
|
||||
Integer getPreferredVersion(int maxVersion);
|
||||
|
||||
|
@ -56,8 +56,8 @@ public interface VersionNegotiator {
|
|||
* Indicates whether or not the specified version is supported by this
|
||||
* resource
|
||||
*
|
||||
* @param version
|
||||
* @return
|
||||
* @param version the version to test
|
||||
* @return true if supported; false otherwise
|
||||
*/
|
||||
boolean isVersionSupported(int version);
|
||||
|
||||
|
|
|
@ -73,7 +73,7 @@ public class CompressionInputStream extends InputStream {
|
|||
fillBuffer(fourByteBuffer);
|
||||
compressedBuffer = new byte[toInt(fourByteBuffer)];
|
||||
|
||||
bufferIndex = buffer.length; // indicate that buffer is empty
|
||||
bufferIndex = buffer.length; // indicate that buffer is empty
|
||||
}
|
||||
|
||||
private int toInt(final byte[] data) {
|
||||
|
@ -175,7 +175,8 @@ public class CompressionInputStream extends InputStream {
|
|||
|
||||
/**
|
||||
* Does nothing. Does NOT close underlying InputStream
|
||||
* @throws java.io.IOException
|
||||
*
|
||||
* @throws java.io.IOException for any issues closing underlying stream
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -61,7 +61,7 @@ public class CompressionOutputStream extends OutputStream {
|
|||
* Compresses the currently buffered chunk of data and sends it to the
|
||||
* output stream
|
||||
*
|
||||
* @throws IOException
|
||||
* @throws IOException if issues occur writing to stream
|
||||
*/
|
||||
protected void compressAndWrite() throws IOException {
|
||||
if (bufferIndex <= 0) {
|
||||
|
|
|
@ -42,7 +42,7 @@ public class BufferStateManager {
|
|||
* resizing the buffer if necessary. This operation MAY change the direction
|
||||
* of the buffer.
|
||||
*
|
||||
* @param requiredSize
|
||||
* @param requiredSize the desired size of the buffer
|
||||
*/
|
||||
public void ensureSize(final int requiredSize) {
|
||||
if (buffer.capacity() < requiredSize) {
|
||||
|
|
|
@ -43,17 +43,17 @@ public class SocketChannelInputStream extends InputStream {
|
|||
public void setTimeout(final int timeoutMillis) {
|
||||
this.timeoutMillis = timeoutMillis;
|
||||
}
|
||||
|
||||
|
||||
public void consume() throws IOException {
|
||||
channel.shutdownInput();
|
||||
|
||||
|
||||
final byte[] b = new byte[4096];
|
||||
final ByteBuffer buffer = ByteBuffer.wrap(b);
|
||||
int bytesRead;
|
||||
do {
|
||||
bytesRead = channel.read(buffer);
|
||||
buffer.flip();
|
||||
} while ( bytesRead > 0 );
|
||||
} while (bytesRead > 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -160,7 +160,8 @@ public class SocketChannelInputStream extends InputStream {
|
|||
|
||||
/**
|
||||
* Closes the underlying socket channel.
|
||||
* @throws java.io.IOException
|
||||
*
|
||||
* @throws java.io.IOException for issues closing underlying stream
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -104,7 +104,8 @@ public class SocketChannelOutputStream extends OutputStream {
|
|||
|
||||
/**
|
||||
* Closes the underlying SocketChannel
|
||||
* @throws java.io.IOException
|
||||
*
|
||||
* @throws java.io.IOException if issues closing underlying stream
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
|
|
@ -260,7 +260,7 @@ public class SSLSocketChannel implements Closeable {
|
|||
|
||||
public void consume() throws IOException {
|
||||
channel.shutdownInput();
|
||||
|
||||
|
||||
final byte[] b = new byte[4096];
|
||||
final ByteBuffer buffer = ByteBuffer.wrap(b);
|
||||
int readCount;
|
||||
|
@ -269,7 +269,7 @@ public class SSLSocketChannel implements Closeable {
|
|||
buffer.flip();
|
||||
} while (readCount > 0);
|
||||
}
|
||||
|
||||
|
||||
private int readData(final ByteBuffer dest) throws IOException {
|
||||
final long startTime = System.currentTimeMillis();
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ public class SSLSocketChannelInputStream extends InputStream {
|
|||
public void consume() throws IOException {
|
||||
channel.consume();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
return channel.read();
|
||||
|
|
|
@ -98,8 +98,7 @@ public class ByteArrayOutputStream extends OutputStream {
|
|||
newCapacity = minCapacity;
|
||||
}
|
||||
if (newCapacity < 0) {
|
||||
if (minCapacity < 0) // overflow
|
||||
{
|
||||
if (minCapacity < 0) { // overflow
|
||||
throw new OutOfMemoryError();
|
||||
}
|
||||
newCapacity = Integer.MAX_VALUE;
|
||||
|
@ -170,8 +169,7 @@ public class ByteArrayOutputStream extends OutputStream {
|
|||
* @return the current contents of this output stream, as a byte array.
|
||||
* @see java.io.ByteArrayOutputStream#size()
|
||||
*/
|
||||
public byte toByteArray ()
|
||||
[] {
|
||||
public byte[] toByteArray() {
|
||||
return Arrays.copyOf(buf, count);
|
||||
}
|
||||
|
||||
|
|
|
@ -39,8 +39,6 @@ public class ByteCountingOutputStream extends OutputStream {
|
|||
write(b, 0, b.length);
|
||||
}
|
||||
|
||||
;
|
||||
|
||||
@Override
|
||||
public void write(byte[] b, int off, int len) throws IOException {
|
||||
out.write(b, off, len);
|
||||
|
|
|
@ -104,7 +104,7 @@ public class LeakyBucketStreamThrottler implements StreamThrottler {
|
|||
|
||||
@Override
|
||||
public int read(final byte[] b) throws IOException {
|
||||
if(b.length == 0){
|
||||
if (b.length == 0) {
|
||||
return 0;
|
||||
}
|
||||
return read(b, 0, b.length);
|
||||
|
@ -112,13 +112,13 @@ public class LeakyBucketStreamThrottler implements StreamThrottler {
|
|||
|
||||
@Override
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
if ( len < 0 ) {
|
||||
if (len < 0) {
|
||||
throw new IllegalArgumentException();
|
||||
}
|
||||
if ( len == 0 ) {
|
||||
if (len == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
baos.reset();
|
||||
final int copied = (int) LeakyBucketStreamThrottler.this.copy(toWrap, baos, len);
|
||||
if (copied == 0) {
|
||||
|
|
|
@ -106,6 +106,6 @@ public class LimitingInputStream extends InputStream {
|
|||
}
|
||||
|
||||
public long getLimit() {
|
||||
return limit;
|
||||
return limit;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,72 +22,71 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* An InputStream that will throw EOFException if the underlying InputStream runs out of data before reaching the
|
||||
* configured minimum amount of data
|
||||
* An InputStream that will throw EOFException if the underlying InputStream
|
||||
* runs out of data before reaching the configured minimum amount of data
|
||||
*/
|
||||
public class MinimumLengthInputStream extends FilterInputStream {
|
||||
|
||||
private final long minLength;
|
||||
private long consumedCount = 0L;
|
||||
|
||||
public MinimumLengthInputStream(final InputStream in, final long minLength) {
|
||||
super(in);
|
||||
this.minLength = minLength;
|
||||
}
|
||||
private final long minLength;
|
||||
private long consumedCount = 0L;
|
||||
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
final int b = super.read();
|
||||
if ( b < 0 && consumedCount < minLength ) {
|
||||
throw new EOFException();
|
||||
}
|
||||
|
||||
if ( b >= 0 ) {
|
||||
consumedCount++;
|
||||
}
|
||||
|
||||
return b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b) throws IOException {
|
||||
return read(b, 0, b.length);
|
||||
}
|
||||
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
final int num = super.read(b, off, len);
|
||||
|
||||
if ( num < 0 && consumedCount < minLength ) {
|
||||
throw new EOFException();
|
||||
}
|
||||
|
||||
if ( num >= 0 ) {
|
||||
consumedCount += num;
|
||||
}
|
||||
public MinimumLengthInputStream(final InputStream in, final long minLength) {
|
||||
super(in);
|
||||
this.minLength = minLength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
final int b = super.read();
|
||||
if (b < 0 && consumedCount < minLength) {
|
||||
throw new EOFException();
|
||||
}
|
||||
|
||||
if (b >= 0) {
|
||||
consumedCount++;
|
||||
}
|
||||
|
||||
return b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b) throws IOException {
|
||||
return read(b, 0, b.length);
|
||||
}
|
||||
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
final int num = super.read(b, off, len);
|
||||
|
||||
if (num < 0 && consumedCount < minLength) {
|
||||
throw new EOFException();
|
||||
}
|
||||
|
||||
if (num >= 0) {
|
||||
consumedCount += num;
|
||||
}
|
||||
|
||||
return num;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long skip(final long n) throws IOException {
|
||||
long skipped = super.skip(n);
|
||||
if (skipped < 1) {
|
||||
final int b = super.read();
|
||||
if (b >= 0) {
|
||||
skipped = 1;
|
||||
}
|
||||
}
|
||||
|
||||
if (skipped < 0 && consumedCount < minLength) {
|
||||
throw new EOFException();
|
||||
}
|
||||
|
||||
if (skipped >= 0) {
|
||||
consumedCount += skipped;
|
||||
}
|
||||
|
||||
return skipped;
|
||||
}
|
||||
|
||||
return num;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long skip(final long n) throws IOException {
|
||||
long skipped = super.skip(n);
|
||||
if ( skipped < 1 ) {
|
||||
final int b = super.read();
|
||||
if ( b >= 0 ) {
|
||||
skipped = 1;
|
||||
}
|
||||
}
|
||||
|
||||
if ( skipped < 0 && consumedCount < minLength ) {
|
||||
throw new EOFException();
|
||||
}
|
||||
|
||||
if ( skipped >= 0 ) {
|
||||
consumedCount += skipped;
|
||||
}
|
||||
|
||||
return skipped;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -44,10 +44,10 @@ public class StreamUtils {
|
|||
* <code>destination</code>. If <code>numBytes</code> are not available from
|
||||
* <code>source</code>, throws EOFException
|
||||
*
|
||||
* @param source
|
||||
* @param destination
|
||||
* @param numBytes
|
||||
* @throws IOException
|
||||
* @param source the source of bytes to copy
|
||||
* @param destination the destination to copy bytes to
|
||||
* @param numBytes the number of bytes to copy
|
||||
* @throws IOException if any issues occur while copying
|
||||
*/
|
||||
public static void copy(final InputStream source, final OutputStream destination, final long numBytes) throws IOException {
|
||||
final byte[] buffer = new byte[8192];
|
||||
|
@ -68,9 +68,9 @@ public class StreamUtils {
|
|||
* byte array. If the InputStream has less data than the given byte array,
|
||||
* throws an EOFException
|
||||
*
|
||||
* @param source
|
||||
* @param destination
|
||||
* @throws IOException
|
||||
* @param source the source to copy bytes from
|
||||
* @param destination the destination to fill
|
||||
* @throws IOException if any issues occur reading bytes
|
||||
*/
|
||||
public static void fillBuffer(final InputStream source, final byte[] destination) throws IOException {
|
||||
fillBuffer(source, destination, true);
|
||||
|
@ -82,12 +82,12 @@ public class StreamUtils {
|
|||
* throws an EOFException if <code>ensureCapacity</code> is true and
|
||||
* otherwise returns the number of bytes copied
|
||||
*
|
||||
* @param source
|
||||
* @param destination
|
||||
* @param source the source to read bytes from
|
||||
* @param destination the destination to fill
|
||||
* @param ensureCapacity whether or not to enforce that the InputStream have
|
||||
* at least as much data as the capacity of the destination byte array
|
||||
* @return
|
||||
* @throws IOException
|
||||
* @return the number of bytes actually filled
|
||||
* @throws IOException if unable to read from the underlying stream
|
||||
*/
|
||||
public static int fillBuffer(final InputStream source, final byte[] destination, final boolean ensureCapacity) throws IOException {
|
||||
int bytesRead = 0;
|
||||
|
@ -114,19 +114,19 @@ public class StreamUtils {
|
|||
* <code>stoppers</code> parameter (returns the byte pattern matched). The
|
||||
* bytes in the stopper will be copied.
|
||||
*
|
||||
* @param in
|
||||
* @param out
|
||||
* @param maxBytes
|
||||
* @param stoppers
|
||||
* @param in the source to read bytes from
|
||||
* @param out the destination to write bytes to
|
||||
* @param maxBytes the max bytes to copy
|
||||
* @param stoppers patterns of bytes which if seen will cause the copy to stop
|
||||
* @return the byte array matched, or null if end of stream was reached
|
||||
* @throws IOException
|
||||
* @throws IOException if issues occur reading or writing bytes to the underlying streams
|
||||
*/
|
||||
public static byte[] copyInclusive(final InputStream in, final OutputStream out, final int maxBytes, final byte[]... stoppers) throws IOException {
|
||||
if (stoppers.length == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final List<NonThreadSafeCircularBuffer> circularBuffers = new ArrayList<NonThreadSafeCircularBuffer>();
|
||||
final List<NonThreadSafeCircularBuffer> circularBuffers = new ArrayList<>();
|
||||
for (final byte[] stopper : stoppers) {
|
||||
circularBuffers.add(new NonThreadSafeCircularBuffer(stopper));
|
||||
}
|
||||
|
@ -157,12 +157,12 @@ public class StreamUtils {
|
|||
* byte pattern matched will NOT be copied to the output and will be un-read
|
||||
* from the input.
|
||||
*
|
||||
* @param in
|
||||
* @param out
|
||||
* @param maxBytes
|
||||
* @param stoppers
|
||||
* @param in the source to read bytes from
|
||||
* @param out the destination to write bytes to
|
||||
* @param maxBytes the maximum number of bytes to copy
|
||||
* @param stoppers byte patterns which will cause the copy to stop if found
|
||||
* @return the byte array matched, or null if end of stream was reached
|
||||
* @throws IOException
|
||||
* @throws IOException for issues reading or writing to underlying streams
|
||||
*/
|
||||
public static byte[] copyExclusive(final InputStream in, final OutputStream out, final int maxBytes, final byte[]... stoppers) throws IOException {
|
||||
if (stoppers.length == 0) {
|
||||
|
@ -171,7 +171,7 @@ public class StreamUtils {
|
|||
|
||||
int longest = 0;
|
||||
NonThreadSafeCircularBuffer longestBuffer = null;
|
||||
final List<NonThreadSafeCircularBuffer> circularBuffers = new ArrayList<NonThreadSafeCircularBuffer>();
|
||||
final List<NonThreadSafeCircularBuffer> circularBuffers = new ArrayList<>();
|
||||
for (final byte[] stopper : stoppers) {
|
||||
final NonThreadSafeCircularBuffer circularBuffer = new NonThreadSafeCircularBuffer(stopper);
|
||||
if (stopper.length > longest) {
|
||||
|
@ -220,9 +220,9 @@ public class StreamUtils {
|
|||
*
|
||||
* If unable to skip that number of bytes, throws EOFException
|
||||
*
|
||||
* @param stream
|
||||
* @param bytesToSkip
|
||||
* @throws IOException
|
||||
* @param stream the stream to skip over
|
||||
* @param bytesToSkip the number of bytes to skip
|
||||
* @throws IOException if any issues reading or skipping underlying stream
|
||||
*/
|
||||
public static void skip(final InputStream stream, final long bytesToSkip) throws IOException {
|
||||
if (bytesToSkip <= 0) {
|
||||
|
|
|
@ -38,7 +38,7 @@ public class NonThreadSafeCircularBuffer {
|
|||
/**
|
||||
* Returns the oldest byte in the buffer
|
||||
*
|
||||
* @return
|
||||
* @return the oldest byte
|
||||
*/
|
||||
public int getOldestByte() {
|
||||
return buffer[insertionPointer];
|
||||
|
|
|
@ -19,24 +19,25 @@ package org.apache.nifi.util;
|
|||
public class EscapeUtils {
|
||||
|
||||
/**
|
||||
* Escapes the specified html by replacing &, <, >, ", ', /
|
||||
* with their corresponding html entity. If html is null, null is returned.
|
||||
*
|
||||
* @param html
|
||||
* @return
|
||||
* Escapes the specified html by replacing &, <, >, ", ',
|
||||
* / with their corresponding html entity. If html is null, null is
|
||||
* returned.
|
||||
*
|
||||
* @param html to escape
|
||||
* @return escaped html
|
||||
*/
|
||||
public static String escapeHtml(String html) {
|
||||
if (html == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
html = html.replace("&", "&");
|
||||
html = html.replace("<", "<");
|
||||
html = html.replace(">", ">");
|
||||
html = html.replace("\"", """);
|
||||
html = html.replace("'", "'");
|
||||
html = html.replace("/", "/");
|
||||
|
||||
|
||||
return html;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,8 +49,8 @@ public class FormatUtils {
|
|||
/**
|
||||
* Formats the specified count by adding commas.
|
||||
*
|
||||
* @param count
|
||||
* @return
|
||||
* @param count the value to add commas to
|
||||
* @return the string representation of the given value with commas included
|
||||
*/
|
||||
public static String formatCount(final long count) {
|
||||
return NumberFormat.getIntegerInstance().format(count);
|
||||
|
@ -59,9 +59,9 @@ public class FormatUtils {
|
|||
/**
|
||||
* Formats the specified duration in 'mm:ss.SSS' format.
|
||||
*
|
||||
* @param sourceDuration
|
||||
* @param sourceUnit
|
||||
* @return
|
||||
* @param sourceDuration the duration to format
|
||||
* @param sourceUnit the unit to interpret the duration
|
||||
* @return representation of the given time data in minutes/seconds
|
||||
*/
|
||||
public static String formatMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) {
|
||||
final long millis = TimeUnit.MILLISECONDS.convert(sourceDuration, sourceUnit);
|
||||
|
@ -72,9 +72,9 @@ public class FormatUtils {
|
|||
/**
|
||||
* Formats the specified duration in 'HH:mm:ss.SSS' format.
|
||||
*
|
||||
* @param sourceDuration
|
||||
* @param sourceUnit
|
||||
* @return
|
||||
* @param sourceDuration the duration to format
|
||||
* @param sourceUnit the unit to interpret the duration
|
||||
* @return representation of the given time data in hours/minutes/seconds
|
||||
*/
|
||||
public static String formatHoursMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) {
|
||||
final long millis = TimeUnit.MILLISECONDS.convert(sourceDuration, sourceUnit);
|
||||
|
|
|
@ -60,12 +60,10 @@ public class NaiveSearchRingBuffer {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the contents of the internal buffer, which represents the last X
|
||||
* @return the contents of the internal buffer, which represents the last X
|
||||
* bytes added to the buffer, where X is the minimum of the number of bytes
|
||||
* added to the buffer or the length of the byte sequence for which we are
|
||||
* looking
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public byte[] getBufferContents() {
|
||||
final int contentLength = Math.min(lookingFor.length, bufferSize);
|
||||
|
@ -78,20 +76,16 @@ public class NaiveSearchRingBuffer {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the oldest byte in the buffer
|
||||
*
|
||||
* @return
|
||||
* @return the oldest byte in the buffer
|
||||
*/
|
||||
public int getOldestByte() {
|
||||
return buffer[insertionPointer];
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if the number of bytes that have been added to
|
||||
* @return <code>true</code> if the number of bytes that have been added to
|
||||
* the buffer is at least equal to the length of the byte sequence for which
|
||||
* we are searching
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public boolean isFilled() {
|
||||
return bufferSize >= buffer.length;
|
||||
|
@ -110,7 +104,7 @@ public class NaiveSearchRingBuffer {
|
|||
* Add the given byte to the buffer and notify whether or not the byte
|
||||
* completes the desired byte sequence.
|
||||
*
|
||||
* @param data
|
||||
* @param data the data to add to the buffer
|
||||
* @return <code>true</code> if this byte completes the byte sequence,
|
||||
* <code>false</code> otherwise.
|
||||
*/
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.nifi.util;
|
|||
/**
|
||||
* A bean that holds a single value of type T.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class ObjectHolder<T> {
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
/**
|
||||
* Thread-safe implementation of a RingBuffer
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class RingBuffer<T> {
|
||||
|
||||
|
@ -46,8 +45,8 @@ public class RingBuffer<T> {
|
|||
* Adds the given value to the RingBuffer and returns the value that was
|
||||
* removed in order to make room.
|
||||
*
|
||||
* @param value
|
||||
* @return
|
||||
* @param value the new value to add
|
||||
* @return value previously in the buffer
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public T add(final T value) {
|
||||
|
@ -135,8 +134,8 @@ public class RingBuffer<T> {
|
|||
/**
|
||||
* Removes all elements from the RingBuffer that match the given filter
|
||||
*
|
||||
* @param filter
|
||||
* @return
|
||||
* @param filter to use for deciding what is removed
|
||||
* @return always zero
|
||||
*/
|
||||
public int removeSelectedElements(final Filter<T> filter) {
|
||||
int count = 0;
|
||||
|
@ -209,7 +208,7 @@ public class RingBuffer<T> {
|
|||
* will skip all remaining elements in the RingBuffer; otherwise, the next
|
||||
* element will be evaluated until all elements have been evaluated.
|
||||
*
|
||||
* @param evaluator
|
||||
* @param evaluator used to evaluate each item in the ring buffer
|
||||
*/
|
||||
public void forEach(final ForEachEvaluator<T> evaluator) {
|
||||
forEach(evaluator, IterationDirection.FORWARD);
|
||||
|
@ -222,7 +221,7 @@ public class RingBuffer<T> {
|
|||
* will skip all remaining elements in the RingBuffer; otherwise, the next
|
||||
* element will be evaluated until all elements have been evaluated.
|
||||
*
|
||||
* @param evaluator
|
||||
* @param evaluator the evaluator
|
||||
* @param iterationDirection the order in which to iterate over the elements
|
||||
* in the RingBuffer
|
||||
*/
|
||||
|
@ -270,7 +269,7 @@ public class RingBuffer<T> {
|
|||
* Defines an interface that can be used to iterate over all of the elements
|
||||
* in the RingBuffer via the {@link #forEach} method
|
||||
*
|
||||
* @param <S>
|
||||
* @param <S> the type to evaluate
|
||||
*/
|
||||
public static interface ForEachEvaluator<S> {
|
||||
|
||||
|
@ -278,8 +277,8 @@ public class RingBuffer<T> {
|
|||
* Evaluates the given element and returns {@code true} if the next
|
||||
* element should be evaluated, {@code false} otherwise
|
||||
*
|
||||
* @param value
|
||||
* @return
|
||||
* @param value the value to evaluate
|
||||
* @return true if should continue evaluating; false otherwise
|
||||
*/
|
||||
boolean evaluate(S value);
|
||||
}
|
||||
|
|
|
@ -55,8 +55,8 @@ public final class StopWatch {
|
|||
/**
|
||||
* Returns the amount of time that the StopWatch was running.
|
||||
*
|
||||
* @param timeUnit
|
||||
* @return
|
||||
* @param timeUnit the unit for which the duration should be reported
|
||||
* @return the duration of the stopwatch in the specified unit
|
||||
*
|
||||
* @throws IllegalStateException if the StopWatch has not been stopped via
|
||||
* {@link #stop()}
|
||||
|
@ -71,8 +71,8 @@ public final class StopWatch {
|
|||
/**
|
||||
* Returns the amount of time that has elapsed since the timer was started.
|
||||
*
|
||||
* @param timeUnit
|
||||
* @return
|
||||
* @param timeUnit the unit for which the elapsed time should be computed
|
||||
* @return the elapsed time in the specified unit
|
||||
*/
|
||||
public long getElapsed(final TimeUnit timeUnit) {
|
||||
return timeUnit.convert(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS);
|
||||
|
|
|
@ -16,12 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.util;
|
||||
|
||||
/**
|
||||
*
|
||||
* @author unattrib
|
||||
* @param <A>
|
||||
* @param <B>
|
||||
*/
|
||||
public class Tuple<A, B> {
|
||||
|
||||
final A key;
|
||||
|
|
|
@ -29,7 +29,7 @@ public class DebugDisabledTimedLock implements DebuggableTimedLock {
|
|||
|
||||
/**
|
||||
*
|
||||
* @return
|
||||
* @return true if lock obtained; false otherwise
|
||||
*/
|
||||
@Override
|
||||
public boolean tryLock() {
|
||||
|
@ -38,9 +38,9 @@ public class DebugDisabledTimedLock implements DebuggableTimedLock {
|
|||
|
||||
/**
|
||||
*
|
||||
* @param timeout
|
||||
* @param timeUnit
|
||||
* @return
|
||||
* @param timeout the duration of time to wait for the lock
|
||||
* @param timeUnit the unit which provides meaning to the duration
|
||||
* @return true if obtained lock in time; false otherwise
|
||||
*/
|
||||
@Override
|
||||
public boolean tryLock(final long timeout, final TimeUnit timeUnit) {
|
||||
|
@ -51,9 +51,6 @@ public class DebugDisabledTimedLock implements DebuggableTimedLock {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public void lock() {
|
||||
lock.lock();
|
||||
|
|
|
@ -44,8 +44,7 @@ public class DebugEnabledTimedLock implements DebuggableTimedLock {
|
|||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return
|
||||
* @return true if lock obtained; false otherwise
|
||||
*/
|
||||
@Override
|
||||
public boolean tryLock() {
|
||||
|
@ -61,10 +60,9 @@ public class DebugEnabledTimedLock implements DebuggableTimedLock {
|
|||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param timeout
|
||||
* @param timeUnit
|
||||
* @return
|
||||
* @param timeout duration to wait for lock
|
||||
* @param timeUnit unit to understand given duration
|
||||
* @return true if lock obtained in time; false otherwise
|
||||
*/
|
||||
@Override
|
||||
public boolean tryLock(final long timeout, final TimeUnit timeUnit) {
|
||||
|
@ -84,9 +82,6 @@ public class DebugEnabledTimedLock implements DebuggableTimedLock {
|
|||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public void lock() {
|
||||
logger.trace("Obtaining Lock {}", name);
|
||||
|
@ -96,8 +91,7 @@ public class DebugEnabledTimedLock implements DebuggableTimedLock {
|
|||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param task
|
||||
* @param task to release the lock for
|
||||
*/
|
||||
@Override
|
||||
public void unlock(final String task) {
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.util.file;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
|
@ -53,7 +52,7 @@ public class FileUtils {
|
|||
/**
|
||||
* Closes the given closeable quietly - no logging, no exceptions...
|
||||
*
|
||||
* @param closeable
|
||||
* @param closeable the thing to close
|
||||
*/
|
||||
public static void closeQuietly(final Closeable closeable) {
|
||||
if (null != closeable) {
|
||||
|
@ -66,9 +65,9 @@ public class FileUtils {
|
|||
}
|
||||
|
||||
/**
|
||||
* Releases the given lock quietly - no logging, no exception
|
||||
* Releases the given lock quietly no logging, no exception
|
||||
*
|
||||
* @param lock
|
||||
* @param lock the lock to release
|
||||
*/
|
||||
public static void releaseQuietly(final FileLock lock) {
|
||||
if (null != lock) {
|
||||
|
@ -98,9 +97,10 @@ public class FileUtils {
|
|||
* Deletes the given file. If the given file exists but could not be deleted
|
||||
* this will be printed as a warning to the given logger
|
||||
*
|
||||
* @param file
|
||||
* @param logger
|
||||
* @return
|
||||
* @param file the file to delete
|
||||
* @param logger the logger to provide logging information to about the
|
||||
* operation
|
||||
* @return true if given file no longer exists
|
||||
*/
|
||||
public static boolean deleteFile(final File file, final Logger logger) {
|
||||
return FileUtils.deleteFile(file, logger, 1);
|
||||
|
@ -110,8 +110,8 @@ public class FileUtils {
|
|||
* Deletes the given file. If the given file exists but could not be deleted
|
||||
* this will be printed as a warning to the given logger
|
||||
*
|
||||
* @param file
|
||||
* @param logger
|
||||
* @param file the file to delete
|
||||
* @param logger the logger to write to
|
||||
* @param attempts indicates how many times an attempt to delete should be
|
||||
* made
|
||||
* @return true if given file no longer exists
|
||||
|
@ -192,9 +192,9 @@ public class FileUtils {
|
|||
* recursive) that match the given filename filter. If any file cannot be
|
||||
* deleted then this is printed at warn to the given logger.
|
||||
*
|
||||
* @param directory
|
||||
* @param directory the directory to scan for files to delete
|
||||
* @param filter if null then no filter is used
|
||||
* @param logger
|
||||
* @param logger the logger to use
|
||||
*/
|
||||
public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger) {
|
||||
FileUtils.deleteFilesInDir(directory, filter, logger, false);
|
||||
|
@ -205,10 +205,10 @@ public class FileUtils {
|
|||
* that match the given filename filter. If any file cannot be deleted then
|
||||
* this is printed at warn to the given logger.
|
||||
*
|
||||
* @param directory
|
||||
* @param directory the directory to scan
|
||||
* @param filter if null then no filter is used
|
||||
* @param logger
|
||||
* @param recurse
|
||||
* @param logger the logger to use
|
||||
* @param recurse indicates whether to recurse subdirectories
|
||||
*/
|
||||
public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse) {
|
||||
FileUtils.deleteFilesInDir(directory, filter, logger, recurse, false);
|
||||
|
@ -219,10 +219,10 @@ public class FileUtils {
|
|||
* that match the given filename filter. If any file cannot be deleted then
|
||||
* this is printed at warn to the given logger.
|
||||
*
|
||||
* @param directory
|
||||
* @param directory the directory to scan
|
||||
* @param filter if null then no filter is used
|
||||
* @param logger
|
||||
* @param recurse
|
||||
* @param logger the logger
|
||||
* @param recurse whether to recurse subdirectories or not
|
||||
* @param deleteEmptyDirectories default is false; if true will delete
|
||||
* directories found that are empty
|
||||
*/
|
||||
|
@ -248,9 +248,9 @@ public class FileUtils {
|
|||
/**
|
||||
* Deletes given files.
|
||||
*
|
||||
* @param files
|
||||
* @param recurse will recurse
|
||||
* @throws IOException
|
||||
* @param files the files to delete
|
||||
* @param recurse will recurse if true; false otherwise
|
||||
* @throws IOException if any issues deleting specified files
|
||||
*/
|
||||
public static void deleteFiles(final Collection<File> files, final boolean recurse) throws IOException {
|
||||
for (final File file : files) {
|
||||
|
@ -352,8 +352,8 @@ public class FileUtils {
|
|||
* Copies the given source file to the given destination file. The given
|
||||
* destination will be overwritten if it already exists.
|
||||
*
|
||||
* @param source
|
||||
* @param destination
|
||||
* @param source the file to copy
|
||||
* @param destination the file to copy to
|
||||
* @param lockInputFile if true will lock input file during copy; if false
|
||||
* will not
|
||||
* @param lockOutputFile if true will lock output file during copy; if false
|
||||
|
@ -369,11 +369,12 @@ public class FileUtils {
|
|||
* indicating the problem.
|
||||
* @return long number of bytes copied
|
||||
* @throws FileNotFoundException if the source file could not be found
|
||||
* @throws IOException
|
||||
* @throws IOException if unable to read or write the underlying streams
|
||||
* @throws SecurityException if a security manager denies the needed file
|
||||
* operations
|
||||
*/
|
||||
public static long copyFile(final File source, final File destination, final boolean lockInputFile, final boolean lockOutputFile, final boolean move, final Logger logger) throws FileNotFoundException, IOException {
|
||||
public static long copyFile(final File source, final File destination, final boolean lockInputFile, final boolean lockOutputFile, final boolean move, final Logger logger)
|
||||
throws FileNotFoundException, IOException {
|
||||
|
||||
FileInputStream fis = null;
|
||||
FileOutputStream fos = null;
|
||||
|
@ -436,16 +437,16 @@ public class FileUtils {
|
|||
* Copies the given source file to the given destination file. The given
|
||||
* destination will be overwritten if it already exists.
|
||||
*
|
||||
* @param source
|
||||
* @param destination
|
||||
* @param source the file to copy from
|
||||
* @param destination the file to copy to
|
||||
* @param lockInputFile if true will lock input file during copy; if false
|
||||
* will not
|
||||
* @param lockOutputFile if true will lock output file during copy; if false
|
||||
* will not
|
||||
* @param logger
|
||||
* @param logger the logger to use
|
||||
* @return long number of bytes copied
|
||||
* @throws FileNotFoundException if the source file could not be found
|
||||
* @throws IOException
|
||||
* @throws IOException if unable to read or write to file
|
||||
* @throws SecurityException if a security manager denies the needed file
|
||||
* operations
|
||||
*/
|
||||
|
|
|
@ -61,8 +61,8 @@ public class SynchronousFileWatcher {
|
|||
* Checks if the file has been updated according to the configured
|
||||
* {@link UpdateMonitor} and resets the state
|
||||
*
|
||||
* @return
|
||||
* @throws IOException
|
||||
* @return true if updated; false otherwise
|
||||
* @throws IOException if failure occurs checking for changes
|
||||
*/
|
||||
public boolean checkAndReset() throws IOException {
|
||||
if (checkUpdateMillis <= 0) { // if checkUpdateMillis <= 0, always check
|
||||
|
|
|
@ -26,8 +26,6 @@ import org.apache.nifi.util.search.ahocorasick.SearchState;
|
|||
* Defines an interface to search for content given a set of search terms. Any
|
||||
* implementation of search must be thread safe.
|
||||
*
|
||||
* @author
|
||||
* @param <T>
|
||||
*/
|
||||
public interface Search<T> {
|
||||
|
||||
|
@ -35,7 +33,7 @@ public interface Search<T> {
|
|||
* Establishes the dictionary of terms which will be searched in subsequent
|
||||
* search calls. This can be called only once
|
||||
*
|
||||
* @param terms
|
||||
* @param terms the terms to create a dictionary of
|
||||
*/
|
||||
void initializeDictionary(Set<SearchTerm<T>> terms);
|
||||
|
||||
|
@ -43,7 +41,7 @@ public interface Search<T> {
|
|||
* Searches the given input stream for matches between the already specified
|
||||
* dictionary and the contents scanned.
|
||||
*
|
||||
* @param haystack
|
||||
* @param haystack the source data to scan for hits
|
||||
* @param findAll if true will find all matches if false will find only the
|
||||
* first match
|
||||
* @return SearchState containing results Map might be empty which indicates
|
||||
|
|
|
@ -22,8 +22,6 @@ import java.util.Arrays;
|
|||
/**
|
||||
* This is an immutable thread safe object representing a search term
|
||||
*
|
||||
* @author
|
||||
* @param <T>
|
||||
*/
|
||||
public class SearchTerm<T> {
|
||||
|
||||
|
@ -34,8 +32,8 @@ public class SearchTerm<T> {
|
|||
/**
|
||||
* Constructs a SearchTerm. Defensively copies the given byte array
|
||||
*
|
||||
* @param bytes
|
||||
* @throws IllegalArgument exception if given bytes are null or 0 length
|
||||
* @param bytes the bytes of the search term
|
||||
* @throws IllegalArgumentException if given bytes are null or 0 length
|
||||
*/
|
||||
public SearchTerm(final byte[] bytes) {
|
||||
this(bytes, true, null);
|
||||
|
@ -46,9 +44,9 @@ public class SearchTerm<T> {
|
|||
* given byte array. If the caller indicates a defensive copy is not
|
||||
* necessary then they must not change the given arrays state any longer
|
||||
*
|
||||
* @param bytes
|
||||
* @param defensiveCopy
|
||||
* @param reference
|
||||
* @param bytes the bytes of the new search term
|
||||
* @param defensiveCopy if true will make a defensive copy; false otherwise
|
||||
* @param reference a holder for an object which can be retrieved when this search term hits
|
||||
*/
|
||||
public SearchTerm(final byte[] bytes, final boolean defensiveCopy, final T reference) {
|
||||
if (bytes == null || bytes.length == 0) {
|
||||
|
@ -84,7 +82,7 @@ public class SearchTerm<T> {
|
|||
/**
|
||||
* Determines if the given window starts with the same bytes as this term
|
||||
*
|
||||
* @param window Current window of bytes from the haystack being evaluated.
|
||||
* @param window bytes from the haystack being evaluated
|
||||
* @param windowLength The length of the window to consider
|
||||
* @return true if this term starts with the same bytes of the given window
|
||||
*/
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.io.OutputStream;
|
|||
import java.nio.file.Path;
|
||||
import java.util.UUID;
|
||||
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestCompoundUpdateMonitor {
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.nio.file.StandardCopyOption;
|
|||
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
public class TestSynchronousFileWatcher {
|
||||
|
||||
@Test
|
||||
|
|
|
@ -36,13 +36,13 @@ public class TestRingBuffer {
|
|||
@Test
|
||||
public void testGetNewestElement() {
|
||||
final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
|
||||
|
||||
for (int i=0; i < 11; i++) {
|
||||
|
||||
for (int i = 0; i < 11; i++) {
|
||||
ringBuffer.add(i);
|
||||
assertEquals(i, ringBuffer.getNewestElement().intValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testAsList() {
|
||||
final RingBuffer<Integer> ringBuffer = new RingBuffer<>(10);
|
||||
|
|
|
@ -40,10 +40,10 @@ public class ClientUtils {
|
|||
/**
|
||||
* Gets the content at the specified URI.
|
||||
*
|
||||
* @param uri
|
||||
* @return
|
||||
* @throws ClientHandlerException
|
||||
* @throws UniformInterfaceException
|
||||
* @param uri the URI to get the content of
|
||||
* @return the client response resulting from getting the content of the URI
|
||||
* @throws ClientHandlerException if issues occur handling the request
|
||||
* @throws UniformInterfaceException if any interface violations occur
|
||||
*/
|
||||
public ClientResponse get(final URI uri) throws ClientHandlerException, UniformInterfaceException {
|
||||
return get(uri, null);
|
||||
|
@ -52,11 +52,11 @@ public class ClientUtils {
|
|||
/**
|
||||
* Gets the content at the specified URI using the given query parameters.
|
||||
*
|
||||
* @param uri
|
||||
* @param queryParams
|
||||
* @return
|
||||
* @throws ClientHandlerException
|
||||
* @throws UniformInterfaceException
|
||||
* @param uri the URI to get the content of
|
||||
* @param queryParams the query parameters to use in the request
|
||||
* @return the client response resulting from getting the content of the URI
|
||||
* @throws ClientHandlerException if issues occur handling the request
|
||||
* @throws UniformInterfaceException if any interface violations occur
|
||||
*/
|
||||
public ClientResponse get(final URI uri, final Map<String, String> queryParams) throws ClientHandlerException, UniformInterfaceException {
|
||||
// perform the request
|
||||
|
@ -73,9 +73,9 @@ public class ClientUtils {
|
|||
/**
|
||||
* Performs a POST using the specified url and entity body.
|
||||
*
|
||||
* @param uri
|
||||
* @param entity
|
||||
* @return
|
||||
* @param uri the URI to post to
|
||||
* @param entity the item to post
|
||||
* @return the client response of the request
|
||||
*/
|
||||
public ClientResponse post(URI uri, Object entity) throws ClientHandlerException, UniformInterfaceException {
|
||||
// get the resource
|
||||
|
@ -93,9 +93,9 @@ public class ClientUtils {
|
|||
/**
|
||||
* Performs a POST using the specified url and form data.
|
||||
*
|
||||
* @param uri
|
||||
* @param formData
|
||||
* @return
|
||||
* @param uri the uri to post to
|
||||
* @param formData the data to post
|
||||
* @return the client reponse of the post
|
||||
*/
|
||||
public ClientResponse post(URI uri, Map<String, String> formData) throws ClientHandlerException, UniformInterfaceException {
|
||||
// convert the form data
|
||||
|
@ -119,10 +119,10 @@ public class ClientUtils {
|
|||
/**
|
||||
* Performs a HEAD request to the specified URI.
|
||||
*
|
||||
* @param uri
|
||||
* @return
|
||||
* @throws ClientHandlerException
|
||||
* @throws UniformInterfaceException
|
||||
* @param uri the uri to request the head of
|
||||
* @return the client response of the request
|
||||
* @throws ClientHandlerException for issues handling the request
|
||||
* @throws UniformInterfaceException for issues with the request
|
||||
*/
|
||||
public ClientResponse head(final URI uri) throws ClientHandlerException, UniformInterfaceException {
|
||||
// perform the request
|
||||
|
|
|
@ -103,6 +103,15 @@
|
|||
</value>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>src/main/asciidoc/asciidoc-mod.css</exclude> <!-- MIT license confirmed. Excluding due to parse error-->
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<artifactId>maven-assembly-plugin</artifactId>
|
||||
<configuration>
|
||||
|
|
|
@ -34,9 +34,7 @@ public class BulletinBoardDTO {
|
|||
private Date generated;
|
||||
|
||||
/**
|
||||
* The bulletins to populate in the bulletin board.
|
||||
*
|
||||
* @return
|
||||
* @return bulletins to populate in the bulletin board
|
||||
*/
|
||||
public List<BulletinDTO> getBulletins() {
|
||||
return bulletins;
|
||||
|
@ -47,9 +45,7 @@ public class BulletinBoardDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* When this bulletin board was generated.
|
||||
*
|
||||
* @return
|
||||
* @return when this bulletin board was generated
|
||||
*/
|
||||
@XmlJavaTypeAdapter(TimeAdapter.class)
|
||||
public Date getGenerated() {
|
||||
|
|
|
@ -40,9 +40,7 @@ public class BulletinDTO {
|
|||
private Date timestamp;
|
||||
|
||||
/**
|
||||
* The id of this message.
|
||||
*
|
||||
* @return
|
||||
* @return id of this message
|
||||
*/
|
||||
public Long getId() {
|
||||
return id;
|
||||
|
@ -53,10 +51,8 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* When clustered, the address of the node from which this bulletin
|
||||
* originated.
|
||||
*
|
||||
* @return
|
||||
* @return When clustered, the address of the node from which this bulletin
|
||||
* originated
|
||||
*/
|
||||
public String getNodeAddress() {
|
||||
return nodeAddress;
|
||||
|
@ -67,9 +63,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The group id of the source component.
|
||||
*
|
||||
* @return
|
||||
* @return group id of the source component
|
||||
*/
|
||||
public String getGroupId() {
|
||||
return groupId;
|
||||
|
@ -80,9 +74,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The category of this message.
|
||||
*
|
||||
* @return
|
||||
* @return category of this message
|
||||
*/
|
||||
public String getCategory() {
|
||||
return category;
|
||||
|
@ -93,9 +85,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The actual message.
|
||||
*
|
||||
* @return
|
||||
* @return actual message
|
||||
*/
|
||||
public String getMessage() {
|
||||
return message;
|
||||
|
@ -106,9 +96,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The id of the source of this message.
|
||||
*
|
||||
* @return
|
||||
* @return id of the source of this message
|
||||
*/
|
||||
public String getSourceId() {
|
||||
return sourceId;
|
||||
|
@ -119,9 +107,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The name of the source of this message.
|
||||
*
|
||||
* @return
|
||||
* @return name of the source of this message
|
||||
*/
|
||||
public String getSourceName() {
|
||||
return sourceName;
|
||||
|
@ -132,9 +118,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The level of this bulletin.
|
||||
*
|
||||
* @return
|
||||
* @return level of this bulletin
|
||||
*/
|
||||
public String getLevel() {
|
||||
return level;
|
||||
|
@ -145,9 +129,7 @@ public class BulletinDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* When this bulletin was generated as a formatted string.
|
||||
*
|
||||
* @return
|
||||
* @return When this bulletin was generated as a formatted string
|
||||
*/
|
||||
@XmlJavaTypeAdapter(TimeAdapter.class)
|
||||
public Date getTimestamp() {
|
||||
|
|
|
@ -33,9 +33,7 @@ public class BulletinQueryDTO {
|
|||
private Integer limit;
|
||||
|
||||
/**
|
||||
* Include bulletins after this id.
|
||||
*
|
||||
* @return
|
||||
* @return Include bulletins after this id
|
||||
*/
|
||||
public Long getAfter() {
|
||||
return after;
|
||||
|
@ -46,9 +44,7 @@ public class BulletinQueryDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* Include bulletin within this group. Supports a regular expression.
|
||||
*
|
||||
* @return
|
||||
* @return Include bulletin within this group. Supports a regular expression
|
||||
*/
|
||||
public String getGroupId() {
|
||||
return groupId;
|
||||
|
@ -59,9 +55,7 @@ public class BulletinQueryDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* Include bulletins that match this message. Supports a regular expression.
|
||||
*
|
||||
* @return
|
||||
* @return Include bulletins that match this message. Supports a regular expression
|
||||
*/
|
||||
public String getMessage() {
|
||||
return message;
|
||||
|
@ -72,9 +66,7 @@ public class BulletinQueryDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* Include bulletins that match this name. Supports a regular expression.
|
||||
*
|
||||
* @return
|
||||
* @return Include bulletins that match this name. Supports a regular expression
|
||||
*/
|
||||
public String getName() {
|
||||
return name;
|
||||
|
@ -85,9 +77,7 @@ public class BulletinQueryDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* Include bulletins that match this id. Supports a source id.
|
||||
*
|
||||
* @return
|
||||
* @return Include bulletins that match this id. Supports a source id
|
||||
*/
|
||||
public String getSourceId() {
|
||||
return sourceId;
|
||||
|
@ -98,9 +88,7 @@ public class BulletinQueryDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The maximum number of bulletins to return.
|
||||
*
|
||||
* @return
|
||||
* @return The maximum number of bulletins to return
|
||||
*/
|
||||
public Integer getLimit() {
|
||||
return limit;
|
||||
|
|
|
@ -33,9 +33,7 @@ public class ClusterDTO {
|
|||
private Date generated;
|
||||
|
||||
/**
|
||||
* The collection of the node DTOs.
|
||||
*
|
||||
* @return
|
||||
* @return collection of the node DTOs
|
||||
*/
|
||||
public Collection<NodeDTO> getNodes() {
|
||||
return nodes;
|
||||
|
@ -46,9 +44,7 @@ public class ClusterDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* Gets the date/time that this report was generated.
|
||||
*
|
||||
* @return
|
||||
* @return the date/time that this report was generated
|
||||
*/
|
||||
@XmlJavaTypeAdapter(TimeAdapter.class)
|
||||
public Date getGenerated() {
|
||||
|
|
|
@ -29,9 +29,7 @@ public class ComponentHistoryDTO {
|
|||
private Map<String, PropertyHistoryDTO> propertyHistory;
|
||||
|
||||
/**
|
||||
* The component id.
|
||||
*
|
||||
* @return
|
||||
* @return component id
|
||||
*/
|
||||
public String getComponentId() {
|
||||
return componentId;
|
||||
|
@ -42,9 +40,7 @@ public class ComponentHistoryDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The history for this components properties.
|
||||
*
|
||||
* @return
|
||||
* @return history for this components properties
|
||||
*/
|
||||
public Map<String, PropertyHistoryDTO> getPropertyHistory() {
|
||||
return propertyHistory;
|
||||
|
|
|
@ -34,9 +34,7 @@ public class ConnectableDTO {
|
|||
private String comments;
|
||||
|
||||
/**
|
||||
* The id of this connectable component.
|
||||
*
|
||||
* @return
|
||||
* @return id of this connectable component
|
||||
*/
|
||||
public String getId() {
|
||||
return id;
|
||||
|
@ -47,9 +45,7 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The type of this connectable component.
|
||||
*
|
||||
* @return
|
||||
* @return type of this connectable component
|
||||
*/
|
||||
public String getType() {
|
||||
return type;
|
||||
|
@ -60,9 +56,7 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The id of the group that this connectable component resides in.
|
||||
*
|
||||
* @return
|
||||
* @return id of the group that this connectable component resides in
|
||||
*/
|
||||
public String getGroupId() {
|
||||
return groupId;
|
||||
|
@ -73,9 +67,7 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The name of this connectable component.
|
||||
*
|
||||
* @return
|
||||
* @return name of this connectable component
|
||||
*/
|
||||
public String getName() {
|
||||
return name;
|
||||
|
@ -86,9 +78,7 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* Used to reflect the current state of this Connectable.
|
||||
*
|
||||
* @return
|
||||
* @return Used to reflect the current state of this Connectable
|
||||
*/
|
||||
public Boolean isRunning() {
|
||||
return running;
|
||||
|
@ -99,10 +89,8 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* If this represents a remote port it is used to indicate whether the
|
||||
* target exists.
|
||||
*
|
||||
* @return
|
||||
* @return If this represents a remote port it is used to indicate whether the
|
||||
* target exists
|
||||
*/
|
||||
public Boolean getExists() {
|
||||
return exists;
|
||||
|
@ -113,10 +101,8 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* If this represents a remote port it is used to indicate whether is it
|
||||
* configured to transmit.
|
||||
*
|
||||
* @return
|
||||
* @return If this represents a remote port it is used to indicate whether is it
|
||||
* configured to transmit
|
||||
*/
|
||||
public Boolean getTransmitting() {
|
||||
return transmitting;
|
||||
|
@ -127,9 +113,7 @@ public class ConnectableDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The comments from this Connectable.
|
||||
*
|
||||
* @return
|
||||
* @return The comments from this Connectable
|
||||
*/
|
||||
public String getComments() {
|
||||
return comments;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue