mirror of https://github.com/apache/nifi.git
NIFI-4118 RethinkDB bundle with put processor
Signed-off-by: James Wing <jvwing@gmail.com> This closes #1942.
This commit is contained in:
parent
9e296830ab
commit
affc88e599
|
@ -1021,6 +1021,12 @@ The following binary components are provided under the Apache Software License v
|
||||||
This product includes software developed at
|
This product includes software developed at
|
||||||
The Apache Software Foundation (http://www.apache.org/).
|
The Apache Software Foundation (http://www.apache.org/).
|
||||||
|
|
||||||
|
(ASLv2) RethinkDB Java Driver
|
||||||
|
The following NOTICE information applies:
|
||||||
|
RethinkDB Language Drivers
|
||||||
|
|
||||||
|
Copyright 2010-2012 RethinkDB
|
||||||
|
|
||||||
(ASLv2) Carrotsearch HPPC
|
(ASLv2) Carrotsearch HPPC
|
||||||
The following NOTICE information applies:
|
The following NOTICE information applies:
|
||||||
HPPC borrowed code, ideas or both from:
|
HPPC borrowed code, ideas or both from:
|
||||||
|
|
|
@ -286,6 +286,11 @@
|
||||||
<artifactId>nifi-ignite-nar</artifactId>
|
<artifactId>nifi-ignite-nar</artifactId>
|
||||||
<type>nar</type>
|
<type>nar</type>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-rethinkdb-nar</artifactId>
|
||||||
|
<type>nar</type>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-avro-nar</artifactId>
|
<artifactId>nifi-avro-nar</artifactId>
|
||||||
|
|
|
@ -0,0 +1,44 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
|
this work for additional information regarding copyright ownership.
|
||||||
|
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
(the "License"); you may not use this file except in compliance with
|
||||||
|
the License. You may obtain a copy of the License at
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<parent>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-rethinkdb-bundle</artifactId>
|
||||||
|
<version>1.4.0-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
|
||||||
|
<artifactId>nifi-rethinkdb-nar</artifactId>
|
||||||
|
<packaging>nar</packaging>
|
||||||
|
<properties>
|
||||||
|
<maven.javadoc.skip>true</maven.javadoc.skip>
|
||||||
|
<source.skip>true</source.skip>
|
||||||
|
</properties>
|
||||||
|
|
||||||
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-standard-services-api-nar</artifactId>
|
||||||
|
<type>nar</type>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-rethinkdb-processors</artifactId>
|
||||||
|
</dependency>
|
||||||
|
</dependencies>
|
||||||
|
|
||||||
|
</project>
|
|
@ -0,0 +1,209 @@
|
||||||
|
|
||||||
|
Apache License
|
||||||
|
Version 2.0, January 2004
|
||||||
|
http://www.apache.org/licenses/
|
||||||
|
|
||||||
|
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||||
|
|
||||||
|
1. Definitions.
|
||||||
|
|
||||||
|
"License" shall mean the terms and conditions for use, reproduction,
|
||||||
|
and distribution as defined by Sections 1 through 9 of this document.
|
||||||
|
|
||||||
|
"Licensor" shall mean the copyright owner or entity authorized by
|
||||||
|
the copyright owner that is granting the License.
|
||||||
|
|
||||||
|
"Legal Entity" shall mean the union of the acting entity and all
|
||||||
|
other entities that control, are controlled by, or are under common
|
||||||
|
control with that entity. For the purposes of this definition,
|
||||||
|
"control" means (i) the power, direct or indirect, to cause the
|
||||||
|
direction or management of such entity, whether by contract or
|
||||||
|
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||||
|
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||||
|
|
||||||
|
"You" (or "Your") shall mean an individual or Legal Entity
|
||||||
|
exercising permissions granted by this License.
|
||||||
|
|
||||||
|
"Source" form shall mean the preferred form for making modifications,
|
||||||
|
including but not limited to software source code, documentation
|
||||||
|
source, and configuration files.
|
||||||
|
|
||||||
|
"Object" form shall mean any form resulting from mechanical
|
||||||
|
transformation or translation of a Source form, including but
|
||||||
|
not limited to compiled object code, generated documentation,
|
||||||
|
and conversions to other media types.
|
||||||
|
|
||||||
|
"Work" shall mean the work of authorship, whether in Source or
|
||||||
|
Object form, made available under the License, as indicated by a
|
||||||
|
copyright notice that is included in or attached to the work
|
||||||
|
(an example is provided in the Appendix below).
|
||||||
|
|
||||||
|
"Derivative Works" shall mean any work, whether in Source or Object
|
||||||
|
form, that is based on (or derived from) the Work and for which the
|
||||||
|
editorial revisions, annotations, elaborations, or other modifications
|
||||||
|
represent, as a whole, an original work of authorship. For the purposes
|
||||||
|
of this License, Derivative Works shall not include works that remain
|
||||||
|
separable from, or merely link (or bind by name) to the interfaces of,
|
||||||
|
the Work and Derivative Works thereof.
|
||||||
|
|
||||||
|
"Contribution" shall mean any work of authorship, including
|
||||||
|
the original version of the Work and any modifications or additions
|
||||||
|
to that Work or Derivative Works thereof, that is intentionally
|
||||||
|
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||||
|
or by an individual or Legal Entity authorized to submit on behalf of
|
||||||
|
the copyright owner. For the purposes of this definition, "submitted"
|
||||||
|
means any form of electronic, verbal, or written communication sent
|
||||||
|
to the Licensor or its representatives, including but not limited to
|
||||||
|
communication on electronic mailing lists, source code control systems,
|
||||||
|
and issue tracking systems that are managed by, or on behalf of, the
|
||||||
|
Licensor for the purpose of discussing and improving the Work, but
|
||||||
|
excluding communication that is conspicuously marked or otherwise
|
||||||
|
designated in writing by the copyright owner as "Not a Contribution."
|
||||||
|
|
||||||
|
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||||
|
on behalf of whom a Contribution has been received by Licensor and
|
||||||
|
subsequently incorporated within the Work.
|
||||||
|
|
||||||
|
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||||
|
this License, each Contributor hereby grants to You a perpetual,
|
||||||
|
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||||
|
copyright license to reproduce, prepare Derivative Works of,
|
||||||
|
publicly display, publicly perform, sublicense, and distribute the
|
||||||
|
Work and such Derivative Works in Source or Object form.
|
||||||
|
|
||||||
|
3. Grant of Patent License. Subject to the terms and conditions of
|
||||||
|
this License, each Contributor hereby grants to You a perpetual,
|
||||||
|
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||||
|
(except as stated in this section) patent license to make, have made,
|
||||||
|
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||||
|
where such license applies only to those patent claims licensable
|
||||||
|
by such Contributor that are necessarily infringed by their
|
||||||
|
Contribution(s) alone or by combination of their Contribution(s)
|
||||||
|
with the Work to which such Contribution(s) was submitted. If You
|
||||||
|
institute patent litigation against any entity (including a
|
||||||
|
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||||
|
or a Contribution incorporated within the Work constitutes direct
|
||||||
|
or contributory patent infringement, then any patent licenses
|
||||||
|
granted to You under this License for that Work shall terminate
|
||||||
|
as of the date such litigation is filed.
|
||||||
|
|
||||||
|
4. Redistribution. You may reproduce and distribute copies of the
|
||||||
|
Work or Derivative Works thereof in any medium, with or without
|
||||||
|
modifications, and in Source or Object form, provided that You
|
||||||
|
meet the following conditions:
|
||||||
|
|
||||||
|
(a) You must give any other recipients of the Work or
|
||||||
|
Derivative Works a copy of this License; and
|
||||||
|
|
||||||
|
(b) You must cause any modified files to carry prominent notices
|
||||||
|
stating that You changed the files; and
|
||||||
|
|
||||||
|
(c) You must retain, in the Source form of any Derivative Works
|
||||||
|
that You distribute, all copyright, patent, trademark, and
|
||||||
|
attribution notices from the Source form of the Work,
|
||||||
|
excluding those notices that do not pertain to any part of
|
||||||
|
the Derivative Works; and
|
||||||
|
|
||||||
|
(d) If the Work includes a "NOTICE" text file as part of its
|
||||||
|
distribution, then any Derivative Works that You distribute must
|
||||||
|
include a readable copy of the attribution notices contained
|
||||||
|
within such NOTICE file, excluding those notices that do not
|
||||||
|
pertain to any part of the Derivative Works, in at least one
|
||||||
|
of the following places: within a NOTICE text file distributed
|
||||||
|
as part of the Derivative Works; within the Source form or
|
||||||
|
documentation, if provided along with the Derivative Works; or,
|
||||||
|
within a display generated by the Derivative Works, if and
|
||||||
|
wherever such third-party notices normally appear. The contents
|
||||||
|
of the NOTICE file are for informational purposes only and
|
||||||
|
do not modify the License. You may add Your own attribution
|
||||||
|
notices within Derivative Works that You distribute, alongside
|
||||||
|
or as an addendum to the NOTICE text from the Work, provided
|
||||||
|
that such additional attribution notices cannot be construed
|
||||||
|
as modifying the License.
|
||||||
|
|
||||||
|
You may add Your own copyright statement to Your modifications and
|
||||||
|
may provide additional or different license terms and conditions
|
||||||
|
for use, reproduction, or distribution of Your modifications, or
|
||||||
|
for any such Derivative Works as a whole, provided Your use,
|
||||||
|
reproduction, and distribution of the Work otherwise complies with
|
||||||
|
the conditions stated in this License.
|
||||||
|
|
||||||
|
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||||
|
any Contribution intentionally submitted for inclusion in the Work
|
||||||
|
by You to the Licensor shall be under the terms and conditions of
|
||||||
|
this License, without any additional terms or conditions.
|
||||||
|
Notwithstanding the above, nothing herein shall supersede or modify
|
||||||
|
the terms of any separate license agreement you may have executed
|
||||||
|
with Licensor regarding such Contributions.
|
||||||
|
|
||||||
|
6. Trademarks. This License does not grant permission to use the trade
|
||||||
|
names, trademarks, service marks, or product names of the Licensor,
|
||||||
|
except as required for reasonable and customary use in describing the
|
||||||
|
origin of the Work and reproducing the content of the NOTICE file.
|
||||||
|
|
||||||
|
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||||
|
agreed to in writing, Licensor provides the Work (and each
|
||||||
|
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||||
|
implied, including, without limitation, any warranties or conditions
|
||||||
|
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||||
|
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||||
|
appropriateness of using or redistributing the Work and assume any
|
||||||
|
risks associated with Your exercise of permissions under this License.
|
||||||
|
|
||||||
|
8. Limitation of Liability. In no event and under no legal theory,
|
||||||
|
whether in tort (including negligence), contract, or otherwise,
|
||||||
|
unless required by applicable law (such as deliberate and grossly
|
||||||
|
negligent acts) or agreed to in writing, shall any Contributor be
|
||||||
|
liable to You for damages, including any direct, indirect, special,
|
||||||
|
incidental, or consequential damages of any character arising as a
|
||||||
|
result of this License or out of the use or inability to use the
|
||||||
|
Work (including but not limited to damages for loss of goodwill,
|
||||||
|
work stoppage, computer failure or malfunction, or any and all
|
||||||
|
other commercial damages or losses), even if such Contributor
|
||||||
|
has been advised of the possibility of such damages.
|
||||||
|
|
||||||
|
9. Accepting Warranty or Additional Liability. While redistributing
|
||||||
|
the Work or Derivative Works thereof, You may choose to offer,
|
||||||
|
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||||
|
or other liability obligations and/or rights consistent with this
|
||||||
|
License. However, in accepting such obligations, You may act only
|
||||||
|
on Your own behalf and on Your sole responsibility, not on behalf
|
||||||
|
of any other Contributor, and only if You agree to indemnify,
|
||||||
|
defend, and hold each Contributor harmless for any liability
|
||||||
|
incurred by, or claims asserted against, such Contributor by reason
|
||||||
|
of your accepting any such warranty or additional liability.
|
||||||
|
|
||||||
|
END OF TERMS AND CONDITIONS
|
||||||
|
|
||||||
|
APPENDIX: How to apply the Apache License to your work.
|
||||||
|
|
||||||
|
To apply the Apache License to your work, attach the following
|
||||||
|
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||||
|
replaced with your own identifying information. (Don't include
|
||||||
|
the brackets!) The text should be enclosed in the appropriate
|
||||||
|
comment syntax for the file format. We also recommend that a
|
||||||
|
file or class name and description of purpose be included on the
|
||||||
|
same "printed page" as the copyright notice for easier
|
||||||
|
identification within third-party archives.
|
||||||
|
|
||||||
|
Copyright [yyyy] [name of copyright owner]
|
||||||
|
|
||||||
|
Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
you may not use this file except in compliance with the License.
|
||||||
|
You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
|
||||||
|
APACHE NIFI SUBCOMPONENTS:
|
||||||
|
|
||||||
|
The Apache NiFi project contains subcomponents with separate copyright
|
||||||
|
notices and license terms. Your use of the source code for the these
|
||||||
|
subcomponents is subject to the terms and conditions of the following
|
||||||
|
licenses.
|
|
@ -0,0 +1,22 @@
|
||||||
|
nifi-rethinkdb-nar
|
||||||
|
Copyright 2016 The Apache Software Foundation
|
||||||
|
|
||||||
|
This product includes software developed at
|
||||||
|
The Apache Software Foundation (http://www.apache.org/).
|
||||||
|
|
||||||
|
===========================================
|
||||||
|
Apache Software License v2
|
||||||
|
===========================================
|
||||||
|
|
||||||
|
The following binary components are provided under the Apache Software License v2
|
||||||
|
|
||||||
|
(ASLv2) RethinkDB Java Driver
|
||||||
|
The following NOTICE information applies:
|
||||||
|
RethinkDB Language Drivers
|
||||||
|
|
||||||
|
Copyright 2010-2012 RethinkDB
|
||||||
|
|
||||||
|
(ASLv2) Apache Commons IO
|
||||||
|
The following NOTICE information applies:
|
||||||
|
Apache Commons IO
|
||||||
|
Copyright 2002-2016 The Apache Software Foundation
|
|
@ -0,0 +1,66 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
|
this work for additional information regarding copyright ownership.
|
||||||
|
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
(the "License"); you may not use this file except in compliance with
|
||||||
|
the License. You may obtain a copy of the License at
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<parent>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-rethinkdb-bundle</artifactId>
|
||||||
|
<version>1.4.0-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
|
||||||
|
<artifactId>nifi-rethinkdb-processors</artifactId>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
|
||||||
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.rethinkdb</groupId>
|
||||||
|
<artifactId>rethinkdb-driver</artifactId>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-api</artifactId>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-utils</artifactId>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>commons-io</groupId>
|
||||||
|
<artifactId>commons-io</artifactId>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-mock</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.slf4j</groupId>
|
||||||
|
<artifactId>slf4j-simple</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.google.guava</groupId>
|
||||||
|
<artifactId>guava</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
</dependencies>
|
||||||
|
</project>
|
|
@ -0,0 +1,170 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.processors.rethinkdb;
|
||||||
|
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||||
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
|
import org.apache.nifi.processor.AbstractProcessor;
|
||||||
|
import org.apache.nifi.processor.DataUnit;
|
||||||
|
import org.apache.nifi.processor.ProcessContext;
|
||||||
|
import org.apache.nifi.processor.Relationship;
|
||||||
|
import org.apache.nifi.processor.util.StandardValidators;
|
||||||
|
import com.rethinkdb.RethinkDB;
|
||||||
|
import com.rethinkdb.gen.ast.Table;
|
||||||
|
import com.rethinkdb.net.Connection;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract base class for RethinkDb processors
|
||||||
|
*/
|
||||||
|
abstract class AbstractRethinkDBProcessor extends AbstractProcessor {
|
||||||
|
|
||||||
|
protected static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-charset")
|
||||||
|
.displayName("Character Set")
|
||||||
|
.description("Specifies the character set of the document data.")
|
||||||
|
.required(true)
|
||||||
|
.defaultValue("UTF-8")
|
||||||
|
.expressionLanguageSupported(true)
|
||||||
|
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-dbname")
|
||||||
|
.displayName("DB Name")
|
||||||
|
.description("RethinkDB database to connect to")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor DB_HOST = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-host")
|
||||||
|
.displayName("Hostname")
|
||||||
|
.description("RethinkDB hostname")
|
||||||
|
.required(true)
|
||||||
|
.defaultValue("localhost")
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor DB_PORT = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-port")
|
||||||
|
.displayName("DB Port")
|
||||||
|
.description("RethinkDB database port to connect to")
|
||||||
|
.required(true)
|
||||||
|
.defaultValue("28015")
|
||||||
|
.addValidator(StandardValidators.PORT_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-username")
|
||||||
|
.displayName("Username")
|
||||||
|
.description("Username for accessing RethinkDB")
|
||||||
|
.required(false)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-password")
|
||||||
|
.displayName("Password")
|
||||||
|
.description("Password for user")
|
||||||
|
.required(false)
|
||||||
|
.sensitive(true)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-table")
|
||||||
|
.displayName("Table name")
|
||||||
|
.description("RethinkDB table to connect to")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
protected static final PropertyDescriptor MAX_DOCUMENTS_SIZE = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-max-document-size")
|
||||||
|
.displayName("Max size of documents in MBs")
|
||||||
|
.description("Maximum size of documents allowed to be posted in one batch")
|
||||||
|
.defaultValue("1 MB")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
|
||||||
|
.description("Sucessful FlowFiles are routed to this relationship").build();
|
||||||
|
|
||||||
|
static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
|
||||||
|
.description("Failed FlowFiles are routed to this relationship").build();
|
||||||
|
|
||||||
|
public static final String RESULT_ERROR_KEY = "errors";
|
||||||
|
public static final String RESULT_DELETED_KEY = "deleted";
|
||||||
|
public static final String RESULT_GENERATED_KEYS_KEY = "generated_keys";
|
||||||
|
public static final String RESULT_INSERTED_KEY = "inserted";
|
||||||
|
public static final String RESULT_REPLACED_KEY = "replaced";
|
||||||
|
public static final String RESULT_SKIPPED_KEY = "skipped";
|
||||||
|
public static final String RESULT_UNCHANGED_KEY = "unchanged";
|
||||||
|
public static final String RESULT_FIRST_ERROR_KEY = "first_error";
|
||||||
|
public static final String RESULT_WARNINGS_KEY = "warnings";
|
||||||
|
|
||||||
|
protected Connection rethinkDbConnection;
|
||||||
|
protected String databaseName;
|
||||||
|
protected String tableName;
|
||||||
|
protected String username;
|
||||||
|
protected String password;
|
||||||
|
protected String hostname;
|
||||||
|
protected Integer port;
|
||||||
|
protected long maxDocumentsSize;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper method to help testability
|
||||||
|
* @return RethinkDB instance
|
||||||
|
*/
|
||||||
|
protected RethinkDB getRethinkDB() {
|
||||||
|
return RethinkDB.r;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the rdbTable
|
||||||
|
*/
|
||||||
|
protected Table getRdbTable() {
|
||||||
|
return getRethinkDB().db(databaseName).table(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@OnScheduled
|
||||||
|
public void onScheduled(final ProcessContext context) {
|
||||||
|
hostname = context.getProperty(DB_HOST).getValue();
|
||||||
|
port = context.getProperty(DB_PORT).asInteger();
|
||||||
|
username = context.getProperty(USERNAME).getValue();
|
||||||
|
password = context.getProperty(PASSWORD).getValue();
|
||||||
|
databaseName = context.getProperty(DB_NAME).getValue();
|
||||||
|
tableName = context.getProperty(TABLE_NAME).getValue();
|
||||||
|
maxDocumentsSize = context.getProperty(MAX_DOCUMENTS_SIZE).asDataSize(DataUnit.B).longValue();
|
||||||
|
|
||||||
|
try {
|
||||||
|
rethinkDbConnection = makeConnection();
|
||||||
|
} catch(Exception e) {
|
||||||
|
getLogger().error("Error while getting connection " + e.getLocalizedMessage(),e);
|
||||||
|
throw new RuntimeException("Error while getting connection" + e.getLocalizedMessage(),e);
|
||||||
|
}
|
||||||
|
getLogger().info("RethinkDb connection created for host {} port {} and db {}",
|
||||||
|
new Object[] {hostname, port,databaseName});
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Connection makeConnection() {
|
||||||
|
return getRethinkDB().connection().hostname(hostname)
|
||||||
|
.port(port).user(username,
|
||||||
|
password).connect();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,237 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.processors.rethinkdb;
|
||||||
|
|
||||||
|
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||||
|
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||||
|
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||||
|
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||||
|
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||||
|
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||||
|
import org.apache.nifi.annotation.documentation.Tags;
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||||
|
import org.apache.nifi.components.AllowableValue;
|
||||||
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
|
import org.apache.nifi.flowfile.FlowFile;
|
||||||
|
import org.apache.nifi.processor.ProcessContext;
|
||||||
|
import org.apache.nifi.processor.ProcessSession;
|
||||||
|
import org.apache.nifi.processor.Relationship;
|
||||||
|
import org.apache.nifi.processor.exception.ProcessException;
|
||||||
|
import org.json.simple.parser.JSONParser;
|
||||||
|
import com.rethinkdb.gen.ast.Insert;
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.nio.charset.Charset;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
|
||||||
|
@EventDriven
|
||||||
|
@SupportsBatching
|
||||||
|
@Tags({"rethinkdb", "stream","insert", "update", "write", "put"})
|
||||||
|
@CapabilityDescription("Processor to write the JSON content of a FlowFile to RethinkDB (https://www.rethinkdb.com/). The flow file should contain either JSON Object an array of JSON documents")
|
||||||
|
@WritesAttributes({
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_ERROR_MESSAGE, description = "RethinkDB error message"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY, description = "Error count while inserting documents"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY, description = "Number of documents deleted"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY, description = "Keys generated on inserting documents"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY, description = "Number of documents inserted"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY, description = "Number of documents replaced"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY, description = "Number of documents skipped because they already existed"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY, description = "Number of documents unchanged since they already existed"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY, description = "First error while inserting documents"),
|
||||||
|
@WritesAttribute(attribute = PutRethinkDB.RETHINKDB_INSERT_RESULT_WARNINGS_KEY, description = "Warning message in case of large number of ids being returned on insertion")
|
||||||
|
})
|
||||||
|
public class PutRethinkDB extends AbstractRethinkDBProcessor {
|
||||||
|
|
||||||
|
public static AllowableValue CONFLICT_STRATEGY_UPDATE = new AllowableValue("update", "Update", "Update the document having same id with new values");
|
||||||
|
public static AllowableValue CONFLICT_STRATEGY_REPLACE = new AllowableValue("replace", "Replace", "Replace the document with having same id new document");
|
||||||
|
public static AllowableValue CONFLICT_STRATEGY_ERROR = new AllowableValue("error", "Error", "Return error if the document with same id exists");
|
||||||
|
|
||||||
|
public static AllowableValue DURABILITY_SOFT = new AllowableValue("soft", "Soft", "Don't save document on disk before ack");
|
||||||
|
public static AllowableValue DURABILITY_HARD = new AllowableValue("hard", "Hard", "Save document on disk before ack");
|
||||||
|
|
||||||
|
protected static final PropertyDescriptor CONFLICT_STRATEGY = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-conflict-strategy")
|
||||||
|
.displayName("Conflict strategy")
|
||||||
|
.description("Conflict strategy to be used in case of inserting existing document.")
|
||||||
|
.required(true)
|
||||||
|
.defaultValue(CONFLICT_STRATEGY_UPDATE.getValue())
|
||||||
|
.allowableValues(CONFLICT_STRATEGY_UPDATE, CONFLICT_STRATEGY_REPLACE, CONFLICT_STRATEGY_ERROR)
|
||||||
|
.expressionLanguageSupported(true)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
protected static final PropertyDescriptor DURABILITY = new PropertyDescriptor.Builder()
|
||||||
|
.name("rethinkdb-durability")
|
||||||
|
.displayName("Durablity of documents")
|
||||||
|
.description("Durability of documents being inserted")
|
||||||
|
.required(true)
|
||||||
|
.defaultValue("hard")
|
||||||
|
.allowableValues(DURABILITY_HARD, DURABILITY_SOFT)
|
||||||
|
.expressionLanguageSupported(true)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
protected String CONFLICT_OPTION_KEY = "conflict";
|
||||||
|
protected String DURABILITY_OPTION_KEY = "durability";
|
||||||
|
|
||||||
|
private static final Set<Relationship> relationships;
|
||||||
|
private static final List<PropertyDescriptor> propertyDescriptors;
|
||||||
|
|
||||||
|
public static final String RETHINKDB_ERROR_MESSAGE = "rethinkdb.error.message";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT = "rethinkdb.insert.result";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_ERROR_KEY = "rethinkdb.insert.errors";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_DELETED_KEY = "rethinkdb.insert.deleted";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY = "rethinkdb.insert.generated_keys";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_INSERTED_KEY = "rethinkdb.insert.inserted";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_REPLACED_KEY = "rethinkdb.insert.replaced";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_SKIPPED_KEY = "rethinkdb.insert.skipped";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_UNCHANGED_KEY = "rethinkdb.insert.unchanged";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY = "rethinkdb.insert.first_error";
|
||||||
|
public static final String RETHINKDB_INSERT_RESULT_WARNINGS_KEY = "rethinkdb.insert.warnings";
|
||||||
|
|
||||||
|
static {
|
||||||
|
final Set<Relationship> tempRelationships = new HashSet<>();
|
||||||
|
tempRelationships.add(REL_SUCCESS);
|
||||||
|
tempRelationships.add(REL_FAILURE);
|
||||||
|
relationships = Collections.unmodifiableSet(tempRelationships);
|
||||||
|
|
||||||
|
final List<PropertyDescriptor> tempDescriptors = new ArrayList<>();
|
||||||
|
tempDescriptors.add(DB_NAME);
|
||||||
|
tempDescriptors.add(DB_HOST);
|
||||||
|
tempDescriptors.add(DB_PORT);
|
||||||
|
tempDescriptors.add(USERNAME);
|
||||||
|
tempDescriptors.add(PASSWORD);
|
||||||
|
tempDescriptors.add(TABLE_NAME);
|
||||||
|
tempDescriptors.add(CHARSET);
|
||||||
|
tempDescriptors.add(CONFLICT_STRATEGY);
|
||||||
|
tempDescriptors.add(DURABILITY);
|
||||||
|
tempDescriptors.add(MAX_DOCUMENTS_SIZE);
|
||||||
|
propertyDescriptors = Collections.unmodifiableList(tempDescriptors);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<Relationship> getRelationships() {
|
||||||
|
return relationships;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
|
return propertyDescriptors;
|
||||||
|
}
|
||||||
|
|
||||||
|
@OnScheduled
|
||||||
|
public void onScheduled(final ProcessContext context) {
|
||||||
|
super.onScheduled(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||||
|
FlowFile flowFile = session.get();
|
||||||
|
if (flowFile == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if ( flowFile.getSize() == 0) {
|
||||||
|
getLogger().error("Empty message");
|
||||||
|
flowFile = session.putAttribute(flowFile, RETHINKDB_ERROR_MESSAGE, "Empty message size " + flowFile.getSize());
|
||||||
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if ( flowFile.getSize() > maxDocumentsSize) {
|
||||||
|
getLogger().error("Message size exceeded {} max allowed is {}", new Object[] { flowFile.getSize(), maxDocumentsSize});
|
||||||
|
flowFile = session.putAttribute(flowFile, RETHINKDB_ERROR_MESSAGE, "Max message size exceeded " + flowFile.getSize());
|
||||||
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Charset charset = Charset.forName(context.getProperty(CHARSET).evaluateAttributeExpressions(flowFile).getValue());
|
||||||
|
String conflictStrategy = context.getProperty(CONFLICT_STRATEGY).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
|
String durability = context.getProperty(DURABILITY).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
|
|
||||||
|
try {
|
||||||
|
long startTimeMillis = System.currentTimeMillis();
|
||||||
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
|
session.exportTo(flowFile, baos);
|
||||||
|
String documents = new String(baos.toByteArray(), charset);
|
||||||
|
JSONParser parser = new JSONParser();
|
||||||
|
Object jsonDocuments = parser.parse(documents);
|
||||||
|
|
||||||
|
Insert insert = getRdbTable().insert(jsonDocuments)
|
||||||
|
.optArg(CONFLICT_OPTION_KEY, conflictStrategy)
|
||||||
|
.optArg(DURABILITY_OPTION_KEY, durability);
|
||||||
|
|
||||||
|
HashMap<String,Object> result = runInsert(insert);
|
||||||
|
final long endTimeMillis = System.currentTimeMillis();
|
||||||
|
getLogger().debug("Json documents {} inserted Result: {}", new Object[] {documents, result});
|
||||||
|
flowFile = populateAttributes(session, flowFile, result);
|
||||||
|
|
||||||
|
if ( (Long)result.get(RESULT_ERROR_KEY) != 0 ) {
|
||||||
|
getLogger().error("There were errors while inserting data documents {} result {}",
|
||||||
|
new Object [] {documents, result});
|
||||||
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
|
} else {
|
||||||
|
session.transfer(flowFile, REL_SUCCESS);
|
||||||
|
session.getProvenanceReporter().send(flowFile,
|
||||||
|
new StringBuilder("rethinkdb://").append(databaseName).append("/").append(tableName).toString(),
|
||||||
|
(endTimeMillis - startTimeMillis));
|
||||||
|
}
|
||||||
|
} catch (Exception exception) {
|
||||||
|
getLogger().error("Failed to insert into RethinkDB due to {}",
|
||||||
|
new Object[]{exception.getLocalizedMessage()}, exception);
|
||||||
|
flowFile = session.putAttribute(flowFile, RETHINKDB_ERROR_MESSAGE, String.valueOf(exception.getMessage()));
|
||||||
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
|
context.yield();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private FlowFile populateAttributes(final ProcessSession session, FlowFile flowFile,
|
||||||
|
HashMap<String, Object> result) {
|
||||||
|
Map<String,String> resultAttributes = new HashMap<>();
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT, result.toString());
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_ERROR_KEY, String.valueOf(result.get(RESULT_ERROR_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_DELETED_KEY, String.valueOf(result.get(RESULT_DELETED_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY, String.valueOf(result.get(RESULT_GENERATED_KEYS_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_INSERTED_KEY, String.valueOf(result.get(RESULT_INSERTED_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_REPLACED_KEY, String.valueOf(result.get(RESULT_REPLACED_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_SKIPPED_KEY, String.valueOf(result.get(RESULT_SKIPPED_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_UNCHANGED_KEY, String.valueOf(result.get(RESULT_UNCHANGED_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY, String.valueOf(result.get(RESULT_FIRST_ERROR_KEY)));
|
||||||
|
resultAttributes.put(RETHINKDB_INSERT_RESULT_WARNINGS_KEY, String.valueOf(result.get(RESULT_WARNINGS_KEY)));
|
||||||
|
flowFile = session.putAllAttributes(flowFile, resultAttributes);
|
||||||
|
return flowFile;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HashMap<String,Object> runInsert(Insert insert) {
|
||||||
|
return insert.run(rethinkDbConnection);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Shutdown client connection
|
||||||
|
*/
|
||||||
|
@OnStopped
|
||||||
|
public void close() {
|
||||||
|
getLogger().info("Closing connection");
|
||||||
|
if ( rethinkDbConnection != null )
|
||||||
|
rethinkDbConnection.close();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,15 @@
|
||||||
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
# contributor license agreements. See the NOTICE file distributed with
|
||||||
|
# this work for additional information regarding copyright ownership.
|
||||||
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
# (the "License"); you may not use this file except in compliance with
|
||||||
|
# the License. You may obtain a copy of the License at
|
||||||
|
#
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
#
|
||||||
|
# Unless required by applicable law or agreed to in writing, software
|
||||||
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
# See the License for the specific language governing permissions and
|
||||||
|
# limitations under the License.
|
||||||
|
org.apache.nifi.processors.rethinkdb.PutRethinkDB
|
|
@ -0,0 +1,216 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.processors.rethinkdb;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.nifi.util.MockFlowFile;
|
||||||
|
import org.apache.nifi.util.TestRunner;
|
||||||
|
import org.apache.nifi.util.TestRunners;
|
||||||
|
import org.json.simple.JSONArray;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Ignore;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import com.rethinkdb.RethinkDB;
|
||||||
|
import com.rethinkdb.net.Connection;
|
||||||
|
|
||||||
|
import net.minidev.json.JSONObject;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Integration test for RethinkDB. Please ensure that the RethinkDB is running
|
||||||
|
* on local host with default port and has database test with table test and user
|
||||||
|
* admin with password admin before running the integration tests or set the attributes in the
|
||||||
|
* test accordingly.
|
||||||
|
*/
|
||||||
|
@Ignore("Comment this out for running tests against a real instance of RethinkDB")
|
||||||
|
public class ITPutRethinkDBTest {
|
||||||
|
private TestRunner runner;
|
||||||
|
private Connection connection;
|
||||||
|
private String dbName = "test";
|
||||||
|
private String dbHost = "localhost";
|
||||||
|
private String dbPort = "28015";
|
||||||
|
private String user = "admin";
|
||||||
|
private String password = "admin";
|
||||||
|
private String table = "test";
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
runner = TestRunners.newTestRunner(PutRethinkDB.class);
|
||||||
|
runner.setProperty(PutRethinkDB.DB_NAME, dbName);
|
||||||
|
runner.setProperty(PutRethinkDB.DB_HOST, dbHost);
|
||||||
|
runner.setProperty(PutRethinkDB.DB_PORT, dbPort);
|
||||||
|
runner.setProperty(PutRethinkDB.USERNAME, user);
|
||||||
|
runner.setProperty(PutRethinkDB.PASSWORD, password);
|
||||||
|
runner.setProperty(PutRethinkDB.TABLE_NAME, table);
|
||||||
|
runner.setProperty(PutRethinkDB.CHARSET, "UTF-8");
|
||||||
|
runner.setProperty(PutRethinkDB.CONFLICT_STRATEGY, PutRethinkDB.CONFLICT_STRATEGY_UPDATE);
|
||||||
|
runner.setProperty(PutRethinkDB.DURABILITY, PutRethinkDB.DURABILITY_HARD);
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 KB");
|
||||||
|
runner.assertValid();
|
||||||
|
|
||||||
|
connection = RethinkDB.r.connection().user(user, password).db(dbName).hostname(dbHost).port(Integer.parseInt(dbPort)).connect();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
runner = null;
|
||||||
|
connection.close();
|
||||||
|
connection = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidSingleMessage() {
|
||||||
|
RethinkDB.r.db(dbName).table(table).delete().run(connection);
|
||||||
|
long count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 0L, count);
|
||||||
|
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("hello", "rethinkdb");
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_SUCCESS, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"1");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
|
||||||
|
count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 1L, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidSingleMessageTwiceConflictUpdate() {
|
||||||
|
RethinkDB.r.db(dbName).table(table).delete().run(connection);
|
||||||
|
long count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 0L, count);
|
||||||
|
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("id", "rethinkdb");
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,false,true);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"1");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
|
||||||
|
flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"1");
|
||||||
|
assertEquals(flowFiles.get(1).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
|
||||||
|
count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 1L, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidSingleMessageTwiceConflictError() {
|
||||||
|
runner.setProperty(PutRethinkDB.CONFLICT_STRATEGY, PutRethinkDB.CONFLICT_STRATEGY_ERROR);
|
||||||
|
RethinkDB.r.db(dbName).table(table).delete().run(connection);
|
||||||
|
long count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 0L, count);
|
||||||
|
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("id", "rethinkdb");
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,false,true);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"1");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
|
||||||
|
flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_FAILURE);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"1");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY));
|
||||||
|
|
||||||
|
count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 1L, count);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidArrayMessage() {
|
||||||
|
RethinkDB.r.db(dbName).table(table).delete().run(connection);
|
||||||
|
long count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 0L, count);
|
||||||
|
|
||||||
|
JSONObject message1 = new JSONObject();
|
||||||
|
message1.put("hello", "rethinkdb");
|
||||||
|
JSONObject message2 = new JSONObject();
|
||||||
|
message2.put("hello2", "rethinkdb2");
|
||||||
|
JSONArray array = new JSONArray();
|
||||||
|
array.add(message1);
|
||||||
|
array.add(message2);
|
||||||
|
byte [] bytes = array.toJSONString().getBytes();
|
||||||
|
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_SUCCESS, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"2");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
|
||||||
|
count = RethinkDB.r.db(dbName).table(table).count().run(connection);
|
||||||
|
assertEquals("Count should be same", 2L, count);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,357 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.processors.rethinkdb;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.nifi.util.MockFlowFile;
|
||||||
|
import org.apache.nifi.util.TestRunner;
|
||||||
|
import org.apache.nifi.util.TestRunners;
|
||||||
|
import org.json.simple.JSONArray;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import com.rethinkdb.gen.ast.Insert;
|
||||||
|
import com.rethinkdb.net.Connection;
|
||||||
|
import net.minidev.json.JSONObject;
|
||||||
|
|
||||||
|
public class TestPutRethinkDB {
|
||||||
|
private TestRunner runner;
|
||||||
|
private PutRethinkDB mockPutRethinkDB;
|
||||||
|
protected HashMap<String,Object> result = new HashMap<>();
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
result.put(PutRethinkDB.RESULT_DELETED_KEY, 0L);
|
||||||
|
result.put(PutRethinkDB.RESULT_ERROR_KEY, 0L);
|
||||||
|
result.put(PutRethinkDB.RESULT_GENERATED_KEYS_KEY,
|
||||||
|
"[\"4056b236-1d8a-492a-b67f-8cf99e363cc6\"]");
|
||||||
|
result.put(PutRethinkDB.RESULT_INSERTED_KEY, 1L);
|
||||||
|
result.put(PutRethinkDB.RESULT_REPLACED_KEY, 0L);
|
||||||
|
result.put(PutRethinkDB.RESULT_SKIPPED_KEY, 0L);
|
||||||
|
result.put(PutRethinkDB.RESULT_UNCHANGED_KEY, 0L);
|
||||||
|
result.put(PutRethinkDB.RESULT_FIRST_ERROR_KEY, "");
|
||||||
|
result.put(PutRethinkDB.RESULT_WARNINGS_KEY, "");
|
||||||
|
mockPutRethinkDB = new PutRethinkDB() {
|
||||||
|
@Override
|
||||||
|
protected Connection makeConnection() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
protected HashMap<String, Object> runInsert(Insert insert) {
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
runner = TestRunners.newTestRunner(mockPutRethinkDB);
|
||||||
|
runner.setProperty(PutRethinkDB.DB_NAME, "test");
|
||||||
|
runner.setProperty(PutRethinkDB.DB_HOST, "host1");
|
||||||
|
runner.setProperty(PutRethinkDB.DB_PORT, "1234");
|
||||||
|
runner.setProperty(PutRethinkDB.USERNAME, "u1");
|
||||||
|
runner.setProperty(PutRethinkDB.PASSWORD, "p1");
|
||||||
|
runner.setProperty(PutRethinkDB.TABLE_NAME, "t1");
|
||||||
|
runner.setProperty(PutRethinkDB.CHARSET, "UTF-8");
|
||||||
|
runner.setProperty(PutRethinkDB.CONFLICT_STRATEGY, PutRethinkDB.CONFLICT_STRATEGY_UPDATE);
|
||||||
|
runner.setProperty(PutRethinkDB.DURABILITY, PutRethinkDB.DURABILITY_HARD);
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 KB");
|
||||||
|
runner.assertValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
runner = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDefaultValid() {
|
||||||
|
runner.assertValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBlankHost() {
|
||||||
|
runner.setProperty(PutRethinkDB.DB_HOST, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEmptyPort() {
|
||||||
|
runner.setProperty(PutRethinkDB.DB_PORT, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEmptyDBName() {
|
||||||
|
runner.setProperty(PutRethinkDB.DB_NAME, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEmptyUsername() {
|
||||||
|
runner.setProperty(PutRethinkDB.USERNAME, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEmptyPassword() {
|
||||||
|
runner.setProperty(PutRethinkDB.PASSWORD, "p1");
|
||||||
|
runner.assertValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCharsetUTF8() {
|
||||||
|
runner.setProperty(PutRethinkDB.CHARSET, "UTF-8");
|
||||||
|
runner.assertValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCharsetBlank() {
|
||||||
|
runner.setProperty(PutRethinkDB.CHARSET, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
@Test
|
||||||
|
public void testZeroMaxDocumentSize() {
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "0");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBlankConflictStrategy() {
|
||||||
|
runner.setProperty(PutRethinkDB.CONFLICT_STRATEGY, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBlankDurability() {
|
||||||
|
runner.setProperty(PutRethinkDB.DURABILITY, "");
|
||||||
|
runner.assertNotValid();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSizeGreaterThanThreshold() {
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 B");
|
||||||
|
runner.assertValid();
|
||||||
|
byte [] bytes = new byte[2];
|
||||||
|
for (int i = 0; i < bytes.length; i++) {
|
||||||
|
bytes[i] = 'a';
|
||||||
|
}
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_FAILURE, 1);
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_FAILURE);
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_ERROR_MESSAGE));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidSingleMessage() {
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 MB");
|
||||||
|
runner.assertValid();
|
||||||
|
result.remove(PutRethinkDB.RESULT_FIRST_ERROR_KEY);
|
||||||
|
result.remove(PutRethinkDB.RESULT_WARNINGS_KEY);
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("hello", "rethinkdb");
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_SUCCESS, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"1");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInsertThrowsException() {
|
||||||
|
mockPutRethinkDB = new PutRethinkDB() {
|
||||||
|
@Override
|
||||||
|
protected Connection makeConnection() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
protected HashMap<String, Object> runInsert(Insert insert) {
|
||||||
|
throw new RuntimeException("testException");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
runner = TestRunners.newTestRunner(mockPutRethinkDB);
|
||||||
|
runner.setProperty(PutRethinkDB.DB_NAME, "test");
|
||||||
|
runner.setProperty(PutRethinkDB.DB_HOST, "host1");
|
||||||
|
runner.setProperty(PutRethinkDB.DB_PORT, "1234");
|
||||||
|
runner.setProperty(PutRethinkDB.USERNAME, "u1");
|
||||||
|
runner.setProperty(PutRethinkDB.PASSWORD, "p1");
|
||||||
|
runner.setProperty(PutRethinkDB.TABLE_NAME, "t1");
|
||||||
|
runner.setProperty(PutRethinkDB.CHARSET, "UTF-8");
|
||||||
|
runner.setProperty(PutRethinkDB.CONFLICT_STRATEGY, PutRethinkDB.CONFLICT_STRATEGY_UPDATE);
|
||||||
|
runner.setProperty(PutRethinkDB.DURABILITY, PutRethinkDB.DURABILITY_HARD);
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 KB");
|
||||||
|
runner.assertValid();
|
||||||
|
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("hello", "rethinkdb");
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_FAILURE, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_FAILURE);
|
||||||
|
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_ERROR_MESSAGE),"testException");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected=AssertionError.class)
|
||||||
|
public void testMakeConnectionThrowsException() {
|
||||||
|
mockPutRethinkDB = new PutRethinkDB() {
|
||||||
|
@Override
|
||||||
|
protected Connection makeConnection() {
|
||||||
|
throw new RuntimeException("testException");
|
||||||
|
}
|
||||||
|
};
|
||||||
|
runner = TestRunners.newTestRunner(mockPutRethinkDB);
|
||||||
|
runner.setProperty(PutRethinkDB.DB_NAME, "test");
|
||||||
|
runner.setProperty(PutRethinkDB.DB_HOST, "host1");
|
||||||
|
runner.setProperty(PutRethinkDB.DB_PORT, "1234");
|
||||||
|
runner.setProperty(PutRethinkDB.USERNAME, "u1");
|
||||||
|
runner.setProperty(PutRethinkDB.PASSWORD, "p1");
|
||||||
|
runner.setProperty(PutRethinkDB.TABLE_NAME, "t1");
|
||||||
|
runner.setProperty(PutRethinkDB.CHARSET, "UTF-8");
|
||||||
|
runner.setProperty(PutRethinkDB.CONFLICT_STRATEGY, PutRethinkDB.CONFLICT_STRATEGY_UPDATE);
|
||||||
|
runner.setProperty(PutRethinkDB.DURABILITY, PutRethinkDB.DURABILITY_HARD);
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 KB");
|
||||||
|
runner.assertValid();
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("hello", "rethinkdb");
|
||||||
|
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMessageError() {
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 MB");
|
||||||
|
runner.assertValid();
|
||||||
|
result.put(PutRethinkDB.RESULT_FIRST_ERROR_KEY,"Error occured");
|
||||||
|
result.put(PutRethinkDB.RESULT_ERROR_KEY,1L);
|
||||||
|
result.put(PutRethinkDB.RESULT_INSERTED_KEY,0L);
|
||||||
|
result.remove(PutRethinkDB.RESULT_WARNINGS_KEY);
|
||||||
|
JSONObject message = new JSONObject();
|
||||||
|
message.put("hello", "rethinkdb");
|
||||||
|
byte [] bytes = message.toJSONString().getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_FAILURE, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_FAILURE);
|
||||||
|
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"1");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"Error occured");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidArrayMessage() {
|
||||||
|
runner.setProperty(PutRethinkDB.MAX_DOCUMENTS_SIZE, "1 MB");
|
||||||
|
runner.assertValid();
|
||||||
|
result.remove(PutRethinkDB.RESULT_FIRST_ERROR_KEY);
|
||||||
|
result.remove(PutRethinkDB.RESULT_WARNINGS_KEY);
|
||||||
|
result.put(PutRethinkDB.RESULT_INSERTED_KEY,2L);
|
||||||
|
JSONObject message1 = new JSONObject();
|
||||||
|
message1.put("hello", "rethinkdb");
|
||||||
|
JSONObject message2 = new JSONObject();
|
||||||
|
message2.put("hello2", "rethinkdb2");
|
||||||
|
JSONArray array = new JSONArray();
|
||||||
|
array.add(message1);
|
||||||
|
array.add(message2);
|
||||||
|
byte [] bytes = array.toJSONString().getBytes();
|
||||||
|
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_SUCCESS, 1);
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_SUCCESS);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), "0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY),"0");
|
||||||
|
assertNotNull(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY));
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY),"2");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY),"0");
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY),"null");
|
||||||
|
}
|
||||||
|
@Test
|
||||||
|
public void testInvalidSingleMessage() {
|
||||||
|
byte [] bytes = "bad json".getBytes();
|
||||||
|
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_FAILURE, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_FAILURE);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_ERROR_MESSAGE), "null");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInvalidEmptySingleMessage() {
|
||||||
|
byte [] bytes = "".getBytes();
|
||||||
|
runner.enqueue(bytes);
|
||||||
|
runner.run(1,true,true);
|
||||||
|
runner.assertAllFlowFilesTransferred(PutRethinkDB.REL_FAILURE, 1);
|
||||||
|
|
||||||
|
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutRethinkDB.REL_FAILURE);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_DELETED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_GENERATED_KEYS_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_INSERTED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_REPLACED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_SKIPPED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_UNCHANGED_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_INSERT_RESULT_FIRST_ERROR_KEY), null);
|
||||||
|
assertEquals(flowFiles.get(0).getAttribute(PutRethinkDB.RETHINKDB_ERROR_MESSAGE), "Empty message size 0");
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,43 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
|
this work for additional information regarding copyright ownership.
|
||||||
|
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
(the "License"); you may not use this file except in compliance with
|
||||||
|
the License. You may obtain a copy of the License at
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<parent>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-nar-bundles</artifactId>
|
||||||
|
<version>1.4.0-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
|
||||||
|
<artifactId>nifi-rethinkdb-bundle</artifactId>
|
||||||
|
<packaging>pom</packaging>
|
||||||
|
|
||||||
|
<modules>
|
||||||
|
<module>nifi-rethinkdb-processors</module>
|
||||||
|
<module>nifi-rethinkdb-nar</module>
|
||||||
|
</modules>
|
||||||
|
|
||||||
|
<dependencyManagement>
|
||||||
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-rethinkdb-processors</artifactId>
|
||||||
|
<version>1.4.0-SNAPSHOT</version>
|
||||||
|
</dependency>
|
||||||
|
</dependencies>
|
||||||
|
</dependencyManagement>
|
||||||
|
|
||||||
|
</project>
|
|
@ -72,6 +72,7 @@
|
||||||
<module>nifi-datadog-bundle</module>
|
<module>nifi-datadog-bundle</module>
|
||||||
<module>nifi-windows-event-log-bundle</module>
|
<module>nifi-windows-event-log-bundle</module>
|
||||||
<module>nifi-ignite-bundle</module>
|
<module>nifi-ignite-bundle</module>
|
||||||
|
<module>nifi-rethinkdb-bundle</module>
|
||||||
<module>nifi-email-bundle</module>
|
<module>nifi-email-bundle</module>
|
||||||
<module>nifi-ranger-bundle</module>
|
<module>nifi-ranger-bundle</module>
|
||||||
<module>nifi-websocket-bundle</module>
|
<module>nifi-websocket-bundle</module>
|
||||||
|
|
13
pom.xml
13
pom.xml
|
@ -241,7 +241,12 @@
|
||||||
<version>1.5.3-M1</version>
|
<version>1.5.3-M1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.ignite</groupId>
|
<groupId>com.rethinkdb</groupId>
|
||||||
|
<artifactId>rethinkdb-driver</artifactId>
|
||||||
|
<version>2.3.3</version>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.ignite</groupId>
|
||||||
<artifactId>ignite-core</artifactId>
|
<artifactId>ignite-core</artifactId>
|
||||||
<version>1.6.0</version>
|
<version>1.6.0</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
@ -1120,6 +1125,12 @@
|
||||||
<version>1.4.0-SNAPSHOT</version>
|
<version>1.4.0-SNAPSHOT</version>
|
||||||
<type>nar</type>
|
<type>nar</type>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
<artifactId>nifi-rethinkdb-nar</artifactId>
|
||||||
|
<version>1.4.0-SNAPSHOT</version>
|
||||||
|
<type>nar</type>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-ignite-nar</artifactId>
|
<artifactId>nifi-ignite-nar</artifactId>
|
||||||
|
|
Loading…
Reference in New Issue