nifi git commit: NIFI-3523: Ensure that if we are unable to read Provenance events on startup that we don't prevent NiFi from starting

2017-03-03 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 4ed64e756 -> 6c44d6be2


NIFI-3523: Ensure that if we are unable to read Provenance events on startup 
that we don't prevent NiFi from starting

This closes #1542


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6c44d6be
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6c44d6be
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6c44d6be

Branch: refs/heads/master
Commit: 6c44d6be2e31eae491f236f3be39eb3309c45cda
Parents: 4ed64e7
Author: Mark Payne 
Authored: Fri Feb 24 11:45:26 2017 -0500
Committer: Oleg Zhurakousky 
Committed: Fri Mar 3 09:18:29 2017 -0500

--
 .../WriteAheadProvenanceRepository.java  |  7 ++-
 .../provenance/lucene/SimpleIndexManager.java|  2 ++
 .../serialization/CompressableRecordReader.java  | 15 ++-
 .../store/WriteAheadStorePartition.java  |  4 ++--
 .../TestPersistentProvenanceRepository.java  | 19 ++-
 5 files changed, 26 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/6c44d6be/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java
index 229a96d..8975028 100644
--- 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java
@@ -141,7 +141,12 @@ public class WriteAheadProvenanceRepository implements 
ProvenanceRepository {
 eventStore.initialize();
 eventIndex.initialize(eventStore);
 
-eventStore.reindexLatestEvents(eventIndex);
+try {
+eventStore.reindexLatestEvents(eventIndex);
+} catch (final Exception e) {
+logger.error("Failed to re-index some of the Provenance Events. It 
is possible that some of the latest "
++ "events will not be available from the Provenance Repository 
when a query is issued.", e);
+}
 }
 
 @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c44d6be/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
index b0b01e5..4d6c11d 100644
--- 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
@@ -318,7 +318,9 @@ public class SimpleIndexManager implements IndexManager {
 
 // This method exists solely for unit testing purposes.
 protected void close(final IndexWriterCount count) throws IOException {
+logger.debug("Closing Index Writer for {}...", 
count.getWriter().getDirectory());
 count.close();
+logger.debug("Finished closing Index Writer for {}...", 
count.getWriter().getDirectory());
 }
 
 protected int getWriterCount() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/6c44d6be/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordReader.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordReader.java
 

[10/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 54987b9..16a6534 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -16,6 +16,32 @@
  */
 package org.apache.nifi.controller.repository;
 
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
@@ -23,6 +49,7 @@ import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.queue.QueueSize;
 import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ContentClaimWriteCache;
 import org.apache.nifi.controller.repository.claim.ResourceClaim;
 import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream;
 import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream;
@@ -53,32 +80,6 @@ import org.apache.nifi.stream.io.StreamUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayInputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
 /**
  * 
  * Provides a ProcessSession that ensures all accesses, changes and transfers
@@ -143,6 +144,7 @@ public final class StandardProcessSession implements 
ProcessSession, ProvenanceE
 private final Map forkEventBuilders = 
new HashMap<>();
 
 private Checkpoint checkpoint = new Checkpoint();
+private final ContentClaimWriteCache claimCache;
 
 public StandardProcessSession(final ProcessContext context) {
 this.context = context;
@@ -180,7 +182,7 @@ public final class StandardProcessSession implements 
ProcessSession, ProvenanceE
 context.getProvenanceRepository(), this);
 this.sessionId = idGenerator.getAndIncrement();
 this.connectableDescription = description;
-
+this.claimCache = new 
ContentClaimWriteCache(context.getContentRepository());
 LOG.trace("Session {} created for {}", this, connectableDescription);
 processingStartTime = System.nanoTime();
 }
@@ -312,6 +314,11 @@ public final class StandardProcessSession implements 
ProcessSession, ProvenanceE
 final long commitStartNanos = System.nanoTime();
 
 resetReadClaim();
+try {
+claimCache.flush();
+} finally {
+claimCache.reset();
+}
 
 final long updateProvenanceStart = System.nanoTime();
 updateProvenanceRepo(checkpoint);
@@ -375,7 +382,7 @@ public final class StandardProcessSession implements 
ProcessSession, ProvenanceE
 updateEventRepository(checkpoint);
 
 final 

[05/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java
new file mode 100644
index 000..eccff2a
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java
@@ -0,0 +1,363 @@
+/*
+ * 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.provenance.schema;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.repository.schema.FieldMapRecord;
+import org.apache.nifi.repository.schema.NamedValue;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+
+public class LookupTableEventRecord implements Record {
+private final RecordSchema schema;
+private final ProvenanceEventRecord event;
+private final long eventId;
+private final Record contentClaimRecord;
+private final Record previousClaimRecord;
+
+private final long eventIdStartOffset;
+private final long startTimeOffset;
+private final Map componentIdMap;
+private final Map componentTypeMap;
+private final Map queueIdMap;
+private final Map eventTypeMap;
+
+public LookupTableEventRecord(final ProvenanceEventRecord event, final 
long eventId, final RecordSchema schema, final RecordSchema contentClaimSchema,
+final RecordSchema previousContentClaimSchema, final long 
eventIdStartOffset, final long startTimeOffset, final Map 
componentIdMap,
+final Map componentTypeMap, final Map queueIdMap, final Map eventTypeMap) {
+this.schema = schema;
+this.event = event;
+this.eventId = eventId;
+this.previousClaimRecord = 
createPreviousContentClaimRecord(previousContentClaimSchema, 
event.getPreviousContentClaimContainer(), 
event.getPreviousContentClaimSection(),
+event.getPreviousContentClaimIdentifier(), 
event.getPreviousContentClaimOffset(), event.getPreviousFileSize());
+this.contentClaimRecord = createContentClaimRecord(contentClaimSchema, 
event.getContentClaimContainer(), event.getContentClaimSection(),
+event.getContentClaimIdentifier(), event.getContentClaimOffset(), 
event.getFileSize());
+
+this.eventIdStartOffset = eventIdStartOffset;
+this.startTimeOffset = startTimeOffset;
+this.componentIdMap = componentIdMap;
+this.componentTypeMap = componentTypeMap;
+this.queueIdMap = queueIdMap;
+this.eventTypeMap = eventTypeMap;
+}
+
+@Override
+public RecordSchema getSchema() {
+return schema;
+}
+
+
+private static Record createPreviousContentClaimRecord(final RecordSchema 
contentClaimSchema, final String container, final String section,
+final String identifier, final Long offset, final Long size) {
+
+if (container == null || section == null || identifier == null) {
+return null;
+}
+
+final Map fieldValues = new HashMap<>();
+fieldValues.put(EventRecordFields.CONTENT_CLAIM_CONTAINER, container);
+

[01/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 8d467f3d1 -> 96ed405d7


http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
new file mode 100644
index 000..c892376
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java
@@ -0,0 +1,538 @@
+/*
+ * 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.provenance.index.lucene;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.RepositoryConfiguration;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.authorization.EventAuthorizer;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.lineage.LineageNode;
+import org.apache.nifi.provenance.lineage.LineageNodeType;
+import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
+import org.apache.nifi.provenance.lucene.IndexManager;
+import org.apache.nifi.provenance.lucene.SimpleIndexManager;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QueryResult;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.search.SearchTerms;
+import org.apache.nifi.provenance.serialization.StorageSummary;
+import org.apache.nifi.provenance.store.ArrayListEventStore;
+import org.apache.nifi.provenance.store.EventStore;
+import org.apache.nifi.provenance.store.StorageResult;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestLuceneEventIndex {
+
+private final AtomicLong idGenerator = new AtomicLong(0L);
+
+@Rule
+public TestName testName = new TestName();
+
+@BeforeClass
+public static void setLogger() {
+System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", 
"DEBUG");
+}
+
+
+@Test(timeout = 5000)
+public void testGetMinimumIdToReindex() throws InterruptedException {
+final RepositoryConfiguration repoConfig = createConfig(1);
+repoConfig.setDesiredIndexSize(1L);
+final IndexManager indexManager = new SimpleIndexManager(repoConfig);
+
+final ArrayListEventStore eventStore = new ArrayListEventStore();
+final LuceneEventIndex index = new LuceneEventIndex(repoConfig, 
indexManager, 20_000, EventReporter.NO_OP);
+index.initialize(eventStore);
+
+for (int i = 0; i < 50_000; i++) {
+final ProvenanceEventRecord event = createEvent("1234");
+final StorageResult storageResult = eventStore.addEvent(event);
+ 

[03/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java
new file mode 100644
index 000..869febf
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.provenance.store.iterator;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.serialization.RecordReader;
+import org.apache.nifi.provenance.store.RecordReaderFactory;
+
+public class SequentialRecordReaderEventIterator implements EventIterator {
+private final Iterator fileIterator;
+private final RecordReaderFactory readerFactory;
+private final long minimumEventId;
+private final int maxAttributeChars;
+
+private boolean closed = false;
+private RecordReader reader;
+
+public SequentialRecordReaderEventIterator(final List filesToRead, 
final RecordReaderFactory readerFactory, final long minimumEventId, final int 
maxAttributeChars) {
+this.fileIterator = filesToRead.iterator();
+this.readerFactory = readerFactory;
+this.minimumEventId = minimumEventId;
+this.maxAttributeChars = maxAttributeChars;
+}
+
+@Override
+public void close() throws IOException {
+closed = true;
+
+if (reader != null) {
+reader.close();
+}
+}
+
+@Override
+public Optional nextEvent() throws IOException {
+if (closed) {
+throw new IOException("EventIterator is already closed");
+}
+
+if (reader == null) {
+if (!rotateReader()) {
+return Optional.empty();
+}
+}
+
+while (true) {
+final ProvenanceEventRecord event = reader.nextRecord();
+if (event == null) {
+if (rotateReader()) {
+continue;
+} else {
+return Optional.empty();
+}
+} else {
+return Optional.of(event);
+}
+}
+}
+
+private boolean rotateReader() throws IOException {
+final boolean readerExists = (reader != null);
+if (readerExists) {
+reader.close();
+}
+
+boolean multipleReadersOpened = false;
+while (true) {
+if (!fileIterator.hasNext()) {
+return false;
+}
+
+final File eventFile = fileIterator.next();
+try {
+reader = readerFactory.newRecordReader(eventFile, 
Collections.emptyList(), maxAttributeChars);
+break;
+} catch (final FileNotFoundException | EOFException e) {
+multipleReadersOpened = true;
+// File may have aged off or was not fully written. Move to 
next file
+continue;
+}
+}
+
+// If this is the first file in our list, the event of interest may 
not be the first event,
+// so skip to the event that we want.
+if (!readerExists && !multipleReadersOpened) {
+reader.skipToEvent(minimumEventId);
+}
+
+return true;
+}
+}


[11/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
NIFI-3356: Initial implementation of writeahead provenance repository
- The idea behind NIFI-3356 was to improve the efficiency and throughput of the 
Provenance Repository, as it is often the bottleneck. While testing the newly 
designed repository,
  a handful of other, fairly minor, changes were made to improve efficiency as 
well, as these came to light when testing the new repository:

- Use a BufferedOutputStream within StandardProcessSession (via a ClaimCache 
abstraction) in order to avoid continually writing to FileOutputStream when 
writing many small FlowFiles
- Updated threading model of MinimalLockingWriteAheadLog - now performs 
serialization outside of lock and writes to a 'synchronized' OutputStream
- Change minimum scheduling period for components from 30 microseconds to 1 
nanosecond. ScheduledExecutor is very inconsistent with timing of task 
scheduling. With the bored.yield.duration
  now present, this value doesn't need to be set to 30 microseconds. This was 
originally done to avoid processors that had no work from dominating the CPU. 
However, now that we will yield
  when processors have no work, this results in slowing down processors that 
are able to perform work.
- Allow nifi.properties to specify multiple directories for FlowFile Repository
- If backpressure is engaged while running a batch of sessions, then stop batch 
processing earlier. This helps FlowFiles to move through the system much more 
smoothly instead of the
  herky-jerky queuing that we previously saw at very high rates of FlowFiles.
- Added NiFi PID to log message when starting nifi. This was simply an update 
to the log message that provides helpful information.

NIFI-3356: Fixed bug in ContentClaimWriteCache that resulted in data corruption 
and fixed bug in RepositoryConfiguration that threw exception if cache warm 
duration was set to empty string

NIFI-3356: Fixed NPE

NIFI-3356: Added debug-level performance monitoring

NIFI-3356: Updates to unit tests that failed after rebasing against master

NIFI-3356: Incorporated PR review feedback

NIFI-3356: Fixed bug where we would delete index directories that are still in 
use; also added additional debug logging and a simple util class that can be 
used to textualize provenance event files - useful in debugging

This closes #1493


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/96ed405d
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/96ed405d
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/96ed405d

Branch: refs/heads/master
Commit: 96ed405d708894ee5400ebbdbf335325219faa09
Parents: 8d467f3
Author: Mark Payne 
Authored: Fri Dec 9 10:52:33 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Wed Feb 22 12:40:06 2017 -0500

--
 .../nifi/provenance/ProvenanceEventRecord.java  |  10 +
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  10 +-
 .../nifi/provenance/ProgressiveResult.java  |  34 +
 .../nifi/provenance/StandardLineageResult.java  |  47 +-
 .../StandardProvenanceEventRecord.java  |  14 +-
 .../nifi/provenance/StandardQueryResult.java|  91 ++-
 .../nifi/provenance/lineage/EventNode.java  |   3 +-
 .../nifi/repository/schema/FieldMapRecord.java  |   5 +
 .../nifi/repository/schema/RecordSchema.java|   2 +-
 .../repository/schema/SchemaRecordReader.java   |  10 +-
 .../repository/schema/SchemaRecordWriter.java   |   6 +-
 .../org/wali/MinimalLockingWriteAheadLog.java   | 165 ++---
 .../wali/TestMinimalLockingWriteAheadLog.java   |  77 +-
 .../src/main/asciidoc/administration-guide.adoc |  88 ++-
 .../org/apache/nifi/controller/Triggerable.java |   2 +-
 .../nifi/provenance/IdentifierLookup.java   |  88 +++
 .../nifi/provenance/ProvenanceRepository.java   |   7 +-
 .../lineage/ComputeLineageResult.java   |   3 +
 .../apache/nifi/provenance/search/Query.java|   4 +
 .../nifi/provenance/search/QueryResult.java |   3 +
 .../provenance/MockProvenanceRepository.java|   2 +-
 .../java/org/apache/nifi/util/MockFlowFile.java |  24 +-
 .../apache/nifi/controller/AbstractPort.java|   2 +-
 .../apache/nifi/controller/StandardFunnel.java  |   2 +-
 .../apache/nifi/controller/FlowController.java  |  40 +-
 .../repository/FileSystemRepository.java|   1 +
 .../repository/StandardProcessSession.java  | 114 ++-
 .../WriteAheadFlowFileRepository.java   |  36 +-
 .../claim/ContentClaimWriteCache.java   | 166 +
 .../repository/schema/ContentClaimFieldMap.java |  33 +
 .../schema/RepositoryRecordUpdate.java  |   6 +-
 .../scheduling/EventDrivenSchedulingAgent.java  |   4 +-
 .../history/ProcessorStatusDescriptor.java  |  48 +-
 .../VolatileComponentStatusRepository.java  |   4 +-
 .../tasks/ContinuallyRunProcessorTask.java  |  16 +-
 .../repository/TestFileSystemRepository.java|  39 +-
 

[09/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java
new file mode 100644
index 000..bb8d52f
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java
@@ -0,0 +1,241 @@
+/*
+ * 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.provenance;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.provenance.schema.EventFieldNames;
+import org.apache.nifi.provenance.schema.EventIdFirstHeaderSchema;
+import org.apache.nifi.provenance.schema.LookupTableEventRecord;
+import org.apache.nifi.provenance.schema.LookupTableEventSchema;
+import org.apache.nifi.provenance.serialization.CompressableRecordWriter;
+import org.apache.nifi.provenance.serialization.StorageSummary;
+import org.apache.nifi.provenance.toc.TocWriter;
+import org.apache.nifi.repository.schema.FieldMapRecord;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.apache.nifi.repository.schema.SchemaRecordWriter;
+import org.apache.nifi.util.timebuffer.LongEntityAccess;
+import org.apache.nifi.util.timebuffer.TimedBuffer;
+import org.apache.nifi.util.timebuffer.TimestampedLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class EventIdFirstSchemaRecordWriter extends CompressableRecordWriter {
+private static final Logger logger = 
LoggerFactory.getLogger(EventIdFirstSchemaRecordWriter.class);
+
+private static final RecordSchema eventSchema = 
LookupTableEventSchema.EVENT_SCHEMA;
+private static final RecordSchema contentClaimSchema = new 
RecordSchema(eventSchema.getField(EventFieldNames.CONTENT_CLAIM).getSubFields());
+private static final RecordSchema previousContentClaimSchema = new 
RecordSchema(eventSchema.getField(EventFieldNames.PREVIOUS_CONTENT_CLAIM).getSubFields());
+private static final RecordSchema headerSchema = 
EventIdFirstHeaderSchema.SCHEMA;
+
+public static final int SERIALIZATION_VERSION = 1;
+public static final String SERIALIZATION_NAME = 
"EventIdFirstSchemaRecordWriter";
+private final IdentifierLookup idLookup;
+
+private final SchemaRecordWriter schemaRecordWriter = new 
SchemaRecordWriter();
+private final AtomicInteger recordCount = new AtomicInteger(0);
+
+private final Map componentIdMap;
+private final Map componentTypeMap;
+private final Map queueIdMap;
+private static final Map eventTypeMap;
+private static final List eventTypeNames;
+
+private static final TimedBuffer serializeTimes = new 
TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
+private static final TimedBuffer lockTimes = new 
TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
+private static final TimedBuffer writeTimes = new 
TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
+private static final TimedBuffer bytesWritten = new 
TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
+private static final AtomicLong totalRecordCount = new AtomicLong(0L);
+
+private long firstEventId;
+private long systemTimeOffset;
+
+static {
+

[06/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.java
new file mode 100644
index 000..18d3860
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.provenance.lucene;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.TopDocs;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface DocumentToEventConverter {
+
+Set convert(TopDocs topDocs, IndexReader 
indexReader) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java
index f84021f..331d141 100644
--- 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java
@@ -21,17 +21,19 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.nifi.provenance.index.EventIndexSearcher;
+import org.apache.nifi.provenance.index.EventIndexWriter;
 
 public interface IndexManager extends Closeable {
-IndexSearcher borrowIndexSearcher(File indexDir) throws IOException;
+EventIndexSearcher borrowIndexSearcher(File indexDir) throws IOException;
 
-IndexWriter borrowIndexWriter(File indexingDirectory) throws IOException;
+EventIndexWriter borrowIndexWriter(File indexDirectory) throws IOException;
 
-void removeIndex(final File indexDirectory);
+boolean removeIndex(final File indexDirectory);
 
-void returnIndexSearcher(File indexDirectory, IndexSearcher searcher);
+void returnIndexSearcher(EventIndexSearcher searcher);
 
-void returnIndexWriter(File indexingDirectory, IndexWriter writer);
+void returnIndexWriter(EventIndexWriter writer, boolean commit, boolean 
isCloseable);
+
+void returnIndexWriter(EventIndexWriter writer);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
index 8d7df8b..514af38 100644
--- 

[07/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
new file mode 100644
index 000..a583403
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
@@ -0,0 +1,737 @@
+/*
+ * 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.provenance.index.lucene;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.nifi.authorization.AccessDeniedException;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.provenance.AsyncLineageSubmission;
+import org.apache.nifi.provenance.AsyncQuerySubmission;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.RepositoryConfiguration;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StandardLineageResult;
+import org.apache.nifi.provenance.StandardQueryResult;
+import org.apache.nifi.provenance.authorization.EventAuthorizer;
+import org.apache.nifi.provenance.authorization.EventTransformer;
+import org.apache.nifi.provenance.index.EventIndex;
+import org.apache.nifi.provenance.index.EventIndexSearcher;
+import org.apache.nifi.provenance.index.EventIndexWriter;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.lineage.LineageComputationType;
+import org.apache.nifi.provenance.lucene.IndexManager;
+import org.apache.nifi.provenance.lucene.LuceneUtil;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.serialization.StorageSummary;
+import org.apache.nifi.provenance.store.EventStore;
+import org.apache.nifi.provenance.util.DirectoryUtils;
+import org.apache.nifi.provenance.util.NamedThreadFactory;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.util.file.FileUtils;
+import org.apache.nifi.util.timebuffer.LongEntityAccess;
+import org.apache.nifi.util.timebuffer.TimedBuffer;
+import org.apache.nifi.util.timebuffer.TimestampedLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class LuceneEventIndex implements EventIndex {
+private static final Logger logger = 
LoggerFactory.getLogger(LuceneEventIndex.class);
+private static final String EVENT_CATEGORY = "Provenance Repository";
+
+public static final int MAX_UNDELETED_QUERY_RESULTS = 10;
+public static final int MAX_DELETE_INDEX_WAIT_SECONDS = 30;
+public static final int MAX_LINEAGE_NODES = 1000;
+public static final int MAX_INDEX_THREADS = 100;
+
+private final ConcurrentMap

[02/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
index 00f4617..48d8e09 100644
--- 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.FileInputStream;
@@ -65,6 +66,8 @@ import org.apache.nifi.authorization.AccessDeniedException;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.provenance.index.EventIndexSearcher;
+import org.apache.nifi.provenance.index.EventIndexWriter;
 import org.apache.nifi.provenance.lineage.EventNode;
 import org.apache.nifi.provenance.lineage.Lineage;
 import org.apache.nifi.provenance.lineage.LineageEdge;
@@ -83,7 +86,6 @@ import org.apache.nifi.provenance.serialization.RecordReaders;
 import org.apache.nifi.provenance.serialization.RecordWriter;
 import org.apache.nifi.provenance.serialization.RecordWriters;
 import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.stream.io.DataOutputStream;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
 import org.junit.After;
@@ -120,7 +122,7 @@ public class TestPersistentProvenanceRepository {
 
 private static RepositoryConfiguration createConfiguration() {
 config = new RepositoryConfiguration();
-config.addStorageDirectory(new File("target/storage/" + 
UUID.randomUUID().toString()));
+config.addStorageDirectory("1", new File("target/storage/" + 
UUID.randomUUID().toString()));
 config.setCompressOnRollover(true);
 config.setMaxEventFileLife(2000L, TimeUnit.SECONDS);
 config.setCompressionBlockBytes(100);
@@ -152,14 +154,15 @@ public class TestPersistentProvenanceRepository {
 final File tempRecordFile = tempFolder.newFile("record.tmp");
 System.out.println("findJournalSizes position 0 = " + 
tempRecordFile.length());
 
-final RecordWriter writer = 
RecordWriters.newSchemaRecordWriter(tempRecordFile, false, false);
+final AtomicLong idGenerator = new AtomicLong(0L);
+final RecordWriter writer = 
RecordWriters.newSchemaRecordWriter(tempRecordFile, idGenerator, false, false);
 writer.writeHeader(12345L);
 writer.flush();
 headerSize = Long.valueOf(tempRecordFile.length()).intValue();
-writer.writeRecord(record, 12345L);
+writer.writeRecord(record);
 writer.flush();
 recordSize = Long.valueOf(tempRecordFile.length()).intValue() - 
headerSize;
-writer.writeRecord(record2, 23456L);
+writer.writeRecord(record2);
 writer.flush();
 recordSize2 = Long.valueOf(tempRecordFile.length()).intValue() - 
headerSize - recordSize;
 writer.close();
@@ -187,34 +190,45 @@ public class TestPersistentProvenanceRepository {
 
 @After
 public void closeRepo() throws IOException {
-if (repo != null) {
-try {
-repo.close();
-} catch (final IOException ioe) {
-}
+if (repo == null) {
+return;
 }
 
+try {
+repo.close();
+} catch (final IOException ioe) {
+}
+
+// Delete all of the storage files. We do this in order to clean up 
the tons of files that
+// we create but also to ensure that we have closed all of the file 
handles. If we leave any
+// streams open, for instance, this will throw an IOException, causing 
our unit test to fail.
 if (config != null) {
-// Delete all of the storage files. We do this in order to clean 
up the tons of files that
-// we create but also to ensure that we have closed all of the 
file handles. If we leave any
-// streams open, for instance, this will throw an IOException, 
causing our unit test to fail.
-for (final File storageDir : 

[04/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java
new file mode 100644
index 000..ba4acea
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java
@@ -0,0 +1,123 @@
+/*
+ * 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.provenance.store;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.authorization.EventAuthorizer;
+import org.apache.nifi.provenance.authorization.EventTransformer;
+import org.apache.nifi.provenance.index.EventIndex;
+import org.apache.nifi.provenance.serialization.StorageSummary;
+
+/**
+ * 
+ * An Event Store is responsible for storing Provenance Events and retrieving 
them at a later time.
+ * 
+ */
+public interface EventStore extends Closeable {
+
+/**
+ * Performs any initialization routines that need to happen before the 
store is used
+ *
+ * @throws IOException if unable to perform initialization
+ */
+void initialize() throws IOException;
+
+/**
+ * Adds the given events to the store. All events will be written to the 
same Storage Location.
+ * I.e., all of the {@link StorageSummary} objects that are provided when 
calling the {@link StorageResult#getStorageLocations()}
+ * method will return the same value for the {@link 
StorageSummary#getStorageLocation()}. Each one, however, will
+ * have a different Event ID and potentially a different Block Index.
+ *
+ * @param events the events to add
+ * @return a mapping of event to the location where it was stored
+ * @throws IOException if unable to add the events
+ */
+StorageResult addEvents(Iterable events) throws 
IOException;
+
+/**
+ * @return the number of bytes occupied by the events in the store
+ * @throws IOException if unable to determine the size of the store
+ */
+long getSize() throws IOException;
+
+/**
+ * @return the largest Event ID that has been written to this store, or -1 
if no events have yet been stored.
+ */
+long getMaxEventId();
+
+/**
+ * Retrieves the event with the given ID
+ *
+ * @param id the ID of the event to retrieve
+ * @return an Optional containing the Event with the given ID, or an empty 
optional if the event cannot be found
+ * @throws IOException if unable to read the event from storage
+ */
+Optional getEvent(long id) throws IOException;
+
+/**
+ * Retrieves up to maxRecords events from the store, starting with the 
event whose ID is equal to firstRecordId. If that
+ * event cannot be found, then the first event will be the oldest event in 
the store whose ID is greater than firstRecordId.
+ * All events will be returned in the order that they were written to the 
store. I.e., all events will have monotonically
+ * increasing Event ID's. No events will be filtered out, since there is 
no EventAuthorizer provided.
+ *
+ * @param firstRecordId the ID of the first event to retrieve
+ * @param maxRecords the maximum number of records to retrieve. The actual 
number of results returned may be less than this.
+ * @return a List of ProvenanceEventRecord's
+ * @throws IOException if unable to retrieve records from the store
+ */
+List getEvents(long firstRecordId, int maxRecords) 
throws IOException;
+
+/**
+ * Retrieves up to maxRecords events from the store, starting with the 
event whose ID is equal to firstRecordId. 

[08/11] nifi git commit: NIFI-3356: Initial implementation of writeahead provenance repository - The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as

2017-02-22 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/96ed405d/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java
new file mode 100644
index 000..051cd1f
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java
@@ -0,0 +1,160 @@
+/*
+ * 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.provenance.index;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.authorization.EventAuthorizer;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.serialization.StorageSummary;
+import org.apache.nifi.provenance.store.EventStore;
+
+/**
+ * An Event Index is responsible for indexing Provenance Events in such a way 
that the index can be quickly
+ * searched to in order to retrieve events of interest.
+ */
+public interface EventIndex extends Closeable {
+
+/**
+ * Initializes the Event Index, providing it access to the Event Store, in 
case it is necessary for performing
+ * initialization tasks
+ *
+ * @param eventStore the EventStore that holds the events that have been 
given to the repository.
+ */
+void initialize(EventStore eventStore);
+
+/**
+ * Adds the given events to the index so that they can be queried later.
+ *
+ * @param events the events to index along with their associated Storage 
Summaries
+ */
+void addEvents(Map events);
+
+/**
+ * Replaces the entries in the appropriate index with the given events
+ *
+ * @param events the events to add or replace along with their associated 
Storage Summaries
+ */
+void reindexEvents(Map events);
+
+/**
+ * @return the number of bytes that are utilized by the Event Index
+ */
+long getSize();
+
+/**
+ * Submits a Query asynchronously and returns a QuerySubmission that can 
be used to obtain the results
+ *
+ * @param query the query to perform
+ * @param authorizer the authorizer to use in order to determine whether 
or not a particular event should be included in the result
+ * @param userId the ID of the user on whose behalf the query is being 
submitted
+ *
+ * @return a QuerySubmission that can be used to retrieve the results later
+ */
+QuerySubmission submitQuery(Query query, EventAuthorizer authorizer, 
String userId);
+
+/**
+ * Asynchronously computes the lineage for the FlowFile that is identified 
by the Provenance Event with the given ID.
+ *
+ * @param eventId the ID of the Provenance Event for which the lineage 
should be calculated
+ * @param user the NiFi user on whose behalf the computing is being 
performed
+ * @param authorizer the authorizer to use in order to determine whether 
or not a particular event should be included in the result
+ *
+ * @return a ComputeLineageSubmission that can be used to retrieve the 
results later
+ */
+ComputeLineageSubmission submitLineageComputation(long eventId, NiFiUser 
user, EventAuthorizer authorizer);
+
+/**
+ * Asynchronously computes the lineage for the FlowFile that has the given 
FlowFile UUID.
+ *
+ * @param flowFileUuid the UUID of the FlowFile for which the lineage 
should be computed
+ * @param user the NiFi user on whose behalf the 

[2/2] nifi git commit: NIFI-3354 Added support for simple AVRO/CSV/JSON transformers that utilize external Schema Added support for simple Key/Value Schema Registry as Controller Service Added support

2017-02-17 Thread ozhurakousky
NIFI-3354 Added support for simple AVRO/CSV/JSON transformers that utilize 
external Schema
Added support for simple Key/Value Schema Registry as Controller Service
Added support for registering multiple schemas as dynamic properties of Schema 
Registry  Controller Service
Added the following 8 processors
- ExtractAvroFieldsViaSchemaRegistry
- TransformAvroToCSVViaSchemaRegistry
- TransformAvroToJsonViaSchemaRegistry
- TransformCSVToAvroViaSchemaRegistry
- TransformCSVToJsonViaSchemaRegistry
- TransformJsonToAvroViaSchemaRegistry
- TransformJsonToCSVViaSchemaRegistry
- UpdateAttributeWithSchemaViaSchemaRegistry

polishing

NIFI-3354 Adding support for HDFS Schema Registry, unions and default values in 
the Avro Schema and NULL columns in the source CSV

NIFI-3354 Adding support for logicalTypes per the Avro 1.7.7 spec

NIFI-3354 polishing and restructuring CSVUtils

NIFI-3354 renamed processors to address PR comment

NIFI-3354 addressed latest PR comments
- removed HDFS-based ControllerService. It will be migrated into a separate 
bundle as a true extension.
- removed UpdateAttribute. . . processor
- added mime.type attribute to all Transform* processors

NIFI-3354 added missing L entries

This closes pr/1436


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6a1854c9
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6a1854c9
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6a1854c9

Branch: refs/heads/master
Commit: 6a1854c9758005a67d5315f31533fdb88ec55b81
Parents: ded18b9
Author: Oleg Zhurakousky 
Authored: Fri Jan 20 10:04:48 2017 -0500
Committer: Oleg Zhurakousky 
Committed: Fri Feb 17 14:32:06 2017 -0500

--
 nifi-assembly/pom.xml   |   5 +
 .../nifi-registry-nar/pom.xml   |  31 ++
 .../src/main/resources/META-INF/LICENSE | 240 +++
 .../src/main/resources/META-INF/NOTICE  |  66 
 .../nifi-registry-processors/pom.xml|  74 +
 .../processors/AbstractCSVTransformer.java  |  57 
 .../processors/AbstractContentTransformer.java  | 101 +++
 .../processors/AbstractTransformer.java |  93 ++
 .../schemaregistry/processors/AvroUtils.java|  67 +
 .../processors/BaseContentTransformer.java  |  51 
 .../processors/BaseTransformer.java | 189 
 .../schemaregistry/processors/CSVUtils.java | 299 +++
 .../processors/ExtractAvroFields.java   | 100 +++
 .../schemaregistry/processors/JsonUtils.java|  74 +
 .../processors/RegistryCommon.java  |  84 ++
 .../processors/TransformAvroToCSV.java  |  57 
 .../processors/TransformAvroToJson.java |  46 +++
 .../processors/TransformCSVToAvro.java  |  80 +
 .../processors/TransformCSVToJson.java  |  80 +
 .../processors/TransformJsonToAvro.java |  45 +++
 .../processors/TransformJsonToCSV.java  |  45 +++
 .../org.apache.nifi.processor.Processor |  21 ++
 .../processors/TransformersTest.java| 188 
 .../expected_ouput_csv/decimal_logicalType.txt  |   1 +
 .../decimal_logicalType_invalid_scale.txt   |   1 +
 ...mal_logicalType_valid_scale_with_default.txt |   1 +
 .../decimal_logicalType_with_default.txt|   1 +
 .../expected_ouput_csv/primitive_types.txt  |   1 +
 .../primitive_types_with_matching_default.txt   |   1 +
 .../union_null_last_field_with_default.txt  |   1 +
 .../union_null_middle_field_with_default.txt|   1 +
 .../expected_ouput_csv/union_with_default.txt   |   1 +
 ...l_logicalType_invalid_scale_with_default.txt |  16 +
 ...mal_logicalType_valid_scale_with_default.txt |  16 +
 ..._logicalType_valid_scale_with_no_default.txt |  15 +
 .../input_avro/primitive_types_no_defaults.txt  |  11 +
 .../primitive_types_union_with_defaults.txt |  11 +
 .../primitive_types_with_matching_default.txt   |  11 +
 .../primitive_types_with_mismatch_default.txt   |  11 +
 .../input_avro/union_and_matching_defaults.txt  |  18 ++
 .../input_avro/union_and_mismatch_defaults.txt  |  18 ++
 .../resources/input_csv/decimal_logicalType.txt |   1 +
 .../decimal_logicalType_missing_value.txt   |   1 +
 .../resources/input_csv/primitive_types.txt |   1 +
 .../primitive_types_with_matching_default.txt   |   1 +
 .../union_null_last_field_with_default.txt  |   1 +
 .../union_null_middle_field_with_default.txt|   1 +
 .../resources/input_csv/union_with_default.txt  |   1 +
 .../input_csv/union_with_missing_value.txt  |   1 +
 .../nifi-registry-service/pom.xml   |  48 +++
 .../schemaregistry/services/SchemaRegistry.java |  46 +++
 .../services/SimpleKeyValueSchemaRegistry.java  |  96 ++
 ...org.apache.nifi.controller.ControllerService |  15 +
 

[1/2] nifi git commit: NIFI-3354 Added support for simple AVRO/CSV/JSON transformers that utilize external Schema Added support for simple Key/Value Schema Registry as Controller Service Added support

2017-02-17 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master ded18b94d -> 6a1854c97


http://git-wip-us.apache.org/repos/asf/nifi/blob/6a1854c9/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
--
diff --git 
a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
 
b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
new file mode 100644
index 000..ba45563
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformAvroToJson.java
@@ -0,0 +1,46 @@
+/*
+ * 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.schemaregistry.processors;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+
+
+@Tags({ "registry", "schema", "avro", "json", "transform" })
+@CapabilityDescription("Transforms AVRO content of the Flow File to JSON using 
the schema provided by the Schema Registry Service.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public final class TransformAvroToJson extends AbstractContentTransformer {
+
+/**
+ *
+ */
+@Override
+protected Map transform(InputStream in, OutputStream out, 
InvocationContextProperties contextProperties, Schema schema) {
+GenericRecord avroRecord = AvroUtils.read(in, schema);
+JsonUtils.write(avroRecord, out);
+return Collections.singletonMap(CoreAttributes.MIME_TYPE.key(), 
"application/json");
+}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6a1854c9/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
--
diff --git 
a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
 
b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
new file mode 100644
index 000..f44e440
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-processors/src/main/java/org/apache/nifi/schemaregistry/processors/TransformCSVToAvro.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.schemaregistry.processors;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import 

nifi git commit: NIFI-3255 removed dependency on session.merge from SplitText

2017-02-17 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master ec868362f -> ded18b94d


NIFI-3255 removed dependency on session.merge from SplitText

NIFI-3255 addressed PR comments

NIFI-3255 fixed linkage for Split creation

This closes #1394


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/ded18b94
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/ded18b94
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/ded18b94

Branch: refs/heads/master
Commit: ded18b94dbcaa91dcad2d6ff995e0a9c1282323b
Parents: ec86836
Author: Oleg Zhurakousky 
Authored: Wed Jan 4 15:37:30 2017 -0500
Committer: Oleg Zhurakousky 
Committed: Fri Feb 17 12:31:21 2017 -0500

--
 .../nifi/processors/standard/SplitText.java | 38 
 1 file changed, 32 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/ded18b94/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
index e57841f..4e62993 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.standard;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,6 +33,7 @@ import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -56,6 +58,7 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.util.TextLineDemarcator;
 import org.apache.nifi.stream.io.util.TextLineDemarcator.OffsetInfo;
@@ -282,8 +285,7 @@ public class SplitText extends AbstractProcessor {
  * it signifies the header information and its contents will be included in
  * each and every computed split.
  */
-private List generateSplitFlowFiles(final String fragmentId, 
FlowFile sourceFlowFile, SplitInfo splitInfo,
-  List 
computedSplitsInfo, ProcessSession processSession){
+private List generateSplitFlowFiles(String fragmentId, FlowFile 
sourceFlowFile, SplitInfo splitInfo, List computedSplitsInfo, 
ProcessSession processSession){
 List splitFlowFiles = new ArrayList<>();
 FlowFile headerFlowFile = null;
 long headerCrlfLength = 0;
@@ -295,19 +297,19 @@ public class SplitText extends AbstractProcessor {
 
 if ((computedSplitsInfo.size() == 0) && (headerFlowFile != null)) {
 FlowFile splitFlowFile = processSession.clone(sourceFlowFile, 0, 
headerFlowFile.getSize() - headerCrlfLength);
-splitFlowFile = SplitText.this.updateAttributes(processSession, 
splitFlowFile, 0, splitFlowFile.getSize(),
+splitFlowFile = this.updateAttributes(processSession, 
splitFlowFile, 0, splitFlowFile.getSize(),
 fragmentId, fragmentIndex++, 0, 
sourceFlowFile.getAttribute(CoreAttributes.FILENAME.key()));
 splitFlowFiles.add(splitFlowFile);
 } else {
 for (SplitInfo computedSplitInfo : computedSplitsInfo) {
-long length = SplitText.this.removeTrailingNewLines ? 
computedSplitInfo.trimmedLength : computedSplitInfo.length;
+long length = this.removeTrailingNewLines ? 
computedSplitInfo.trimmedLength : computedSplitInfo.length;
 boolean proceedWithClone = headerFlowFile != null || length > 
0;
 if (proceedWithClone) {
 FlowFile splitFlowFile = null;
 if (headerFlowFile != null) {
 if (length > 0) {
 

nifi git commit: NIFI-3290 Reporting task to send bulletins with S2S

2017-01-30 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 89f1bd318 -> 78a0e1e18


NIFI-3290 Reporting task to send bulletins with S2S

This closes #1401


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/78a0e1e1
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/78a0e1e1
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/78a0e1e1

Branch: refs/heads/master
Commit: 78a0e1e18b9d40a10bac4612a9a0ebdc4b7d13bb
Parents: 89f1bd3
Author: Pierre Villard 
Authored: Thu Jan 5 23:08:55 2017 +0100
Committer: Oleg Zhurakousky 
Committed: Mon Jan 30 09:03:35 2017 -0500

--
 .../SiteToSiteBulletinReportingTask.java| 223 +++
 .../SiteToSiteProvenanceReportingTask.java  |   2 +-
 .../org.apache.nifi.reporting.ReportingTask |   3 +-
 .../TestSiteToSiteBulletinReportingTask.java| 199 +
 4 files changed, 425 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/78a0e1e1/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
--
diff --git 
a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
 
b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
new file mode 100644
index 000..9d9b1b7
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteBulletinReportingTask.java
@@ -0,0 +1,223 @@
+/*
+ * 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.reporting;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.OptionalLong;
+import java.util.TimeZone;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import javax.json.Json;
+import javax.json.JsonArray;
+import javax.json.JsonArrayBuilder;
+import javax.json.JsonBuilderFactory;
+import javax.json.JsonObject;
+import javax.json.JsonObjectBuilder;
+
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.configuration.DefaultSchedule;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+
+@Tags({"bulletin", "site", "site to site", "restricted"})
+@CapabilityDescription("Publishes Bulletin events using the Site To Site 
protocol. Note: only up to 5 bulletins are stored per component and up to "
++ "10 bulletins at controller level for a duration of up to 5 minutes. 
If this reporting task is not scheduled frequently enough some bulletins "
++ "may not be sent.")
+@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's 
last bulletin ID so that on restart the task knows where it left off.")
+@Restricted("Provides operator the ability to send sensitive details contained 
in bulletin events to any external system.")
+@DefaultSchedule(strategy = SchedulingStrategy.TIMER_DRIVEN, period = "1 min")
+public class SiteToSiteBulletinReportingTask extends 
AbstractSiteToSiteReportingTask {
+
+static final String 

nifi git commit: NIFI-957 Added the possibility to use DefaultSchedule annotation in reporting tasks

2017-01-30 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 67e245966 -> 89f1bd318


NIFI-957 Added the possibility to use DefaultSchedule annotation in reporting 
tasks

This closes #1400


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/89f1bd31
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/89f1bd31
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/89f1bd31

Branch: refs/heads/master
Commit: 89f1bd3189970f9930bfdd4861fc2043dfd30c90
Parents: 67e2459
Author: Pierre Villard 
Authored: Thu Jan 5 23:14:37 2017 +0100
Committer: Oleg Zhurakousky 
Committed: Mon Jan 30 08:35:39 2017 -0500

--
 .../configuration/DefaultSchedule.java  |  7 ++--
 .../reporting/AbstractReportingTaskNode.java| 24 ++
 .../controller/DummyScheduledReportingTask.java | 34 
 .../nifi/controller/TestFlowController.java | 10 --
 4 files changed, 70 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/89f1bd31/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java
--
diff --git 
a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java
 
b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java
index ff9125c..8635a74 100644
--- 
a/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java
+++ 
b/nifi-api/src/main/java/org/apache/nifi/annotation/configuration/DefaultSchedule.java
@@ -28,8 +28,9 @@ import java.lang.annotation.Inherited;
 
 /**
  * 
- * Marker interface that a Processor can use to configure the schedule 
strategy, the  period and the number of concurrent tasks.
- * Note that the number of Concurrent tasks will be ignored if the annotion 
@TriggerSerialy is used
+ * Marker interface that a Processor can use to configure default settings for 
the schedule strategy, the period and the number of concurrent tasks.
+ * Marker interface that a ReportingTask can use to configure default settings 
the  schedule strategy and the period.
+ * Note that the number of Concurrent tasks will be ignored if the annotation 
@TriggerSerialy is used
  * 
  */
 @Documented
@@ -42,4 +43,4 @@ public @interface DefaultSchedule {
 String period() default "0 sec";
 int concurrentTasks() default 1;
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/89f1bd31/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index deca385..d7ae309 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller.reporting;
 
+import org.apache.nifi.annotation.configuration.DefaultSchedule;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractConfiguredComponent;
 import org.apache.nifi.controller.ConfigurationContext;
@@ -40,8 +41,14 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.core.annotation.AnnotationUtils;
+
 public abstract class AbstractReportingTaskNode extends 
AbstractConfiguredComponent implements ReportingTaskNode {
 
+private static final Logger LOG = 
LoggerFactory.getLogger(AbstractReportingTaskNode.class);
+
 private final ReportingTask reportingTask;
 private final ProcessScheduler processScheduler;
 private final ControllerServiceLookup serviceLookup;
@@ -72,6 +79,23 @@ public abstract class AbstractReportingTaskNode extends 
AbstractConfiguredCompon
 this.reportingTask = reportingTask;
 this.processScheduler = processScheduler;
 this.serviceLookup = controllerServiceProvider;
+
+final Class reportingClass = reportingTask.getClass();
+
+DefaultSchedule dsc = AnnotationUtils.findAnnotation(reportingClass, 

nifi git commit: NIFI-3314 Adjusting Dockerfile for Docker Hub to use defaults for ARGs such that it does not need supporting build script. Making use of openjdk base image as the java image has been

2017-01-29 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 63c763885 -> 67e245966


NIFI-3314 Adjusting Dockerfile for Docker Hub to use defaults for ARGs
such that it does not need supporting build script.  Making use of
openjdk base image as the java image has been deprecated.

This closes #1419


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/67e24596
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/67e24596
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/67e24596

Branch: refs/heads/master
Commit: 67e2459660c7eddac35e8f55e7e5955e6a329bbb
Parents: 63c7638
Author: Aldrin Piri 
Authored: Fri Jan 13 16:56:49 2017 -0500
Committer: Oleg Zhurakousky 
Committed: Sun Jan 29 11:16:22 2017 -0500

--
 nifi-docker/dockerhub/Dockerfile   | 8 
 nifi-docker/dockermaven/Dockerfile | 6 +++---
 2 files changed, 7 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/67e24596/nifi-docker/dockerhub/Dockerfile
--
diff --git a/nifi-docker/dockerhub/Dockerfile b/nifi-docker/dockerhub/Dockerfile
index ae273cc..cd7aaeb 100644
--- a/nifi-docker/dockerhub/Dockerfile
+++ b/nifi-docker/dockerhub/Dockerfile
@@ -16,12 +16,12 @@
 # under the License.
 #
 
-FROM java:8
+FROM openjdk:8
 MAINTAINER Apache NiFi 
 
-ARG UID
-ARG GID
-ARG NIFI_VERSION
+ARG UID=1000
+ARG GID=50
+ARG NIFI_VERSION=1.2.0
 
 ENV NIFI_BASE_DIR /opt/nifi
 ENV NIFI_HOME $NIFI_BASE_DIR/nifi-$NIFI_VERSION

http://git-wip-us.apache.org/repos/asf/nifi/blob/67e24596/nifi-docker/dockermaven/Dockerfile
--
diff --git a/nifi-docker/dockermaven/Dockerfile 
b/nifi-docker/dockermaven/Dockerfile
index 32bb638..bb56b96 100644
--- a/nifi-docker/dockermaven/Dockerfile
+++ b/nifi-docker/dockermaven/Dockerfile
@@ -16,11 +16,11 @@
 # under the License.
 #
 
-FROM java:8
+FROM openjdk:8
 MAINTAINER Apache NiFi 
 
-ARG UID
-ARG GID
+ARG UID=1000
+ARG GID=50
 ARG NIFI_VERSION
 ARG NIFI_BINARY
 



nifi git commit: NIFI-3363: PutKafka NPE with User-Defined partition

2017-01-27 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master f8f66fa22 -> 63c763885


NIFI-3363: PutKafka NPE with User-Defined partition

- Marked PutKafka Partition Strategy property as deprecated, as Kafka 0.8 
client doesn't use 'partitioner.class' as producer property, we don't have to 
specify it.
- Changed Partition Strategy property from a required one to a dynamic 
property, so that existing processor config can stay in valid state.
- Fixed partition property to work.
- Route a flow file if it failed to be published due to invalid partition.

This closes #1425


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/63c76388
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/63c76388
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/63c76388

Branch: refs/heads/master
Commit: 63c763885c36ab06111edf2c9d7743563ea57fcb
Parents: f8f66fa
Author: Koji Kawamura 
Authored: Wed Jan 18 16:48:09 2017 +0900
Committer: Oleg Zhurakousky 
Committed: Fri Jan 27 12:48:23 2017 -0500

--
 .../apache/nifi/processors/kafka/PutKafka.java  | 77 +---
 .../nifi/processors/kafka/PutKafkaTest.java | 57 +++
 2 files changed, 93 insertions(+), 41 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/63c76388/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index ab0618b..616c6f3 100644
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -40,8 +39,6 @@ import 
org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
@@ -97,11 +94,20 @@ public class PutKafka extends 
AbstractKafkaProcessor {
 public static final AllowableValue COMPRESSION_CODEC_SNAPPY = new 
AllowableValue("snappy", "Snappy",
 "Compress messages using Snappy");
 
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property.
+ */
 static final AllowableValue ROUND_ROBIN_PARTITIONING = new 
AllowableValue("Round Robin", "Round Robin",
 "Messages will be assigned partitions in a round-robin fashion, 
sending the first message to Partition 1, "
 + "the next Partition to Partition 2, and so on, wrapping 
as necessary.");
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property.
+ */
 static final AllowableValue RANDOM_PARTITIONING = new 
AllowableValue("Random Robin", "Random",
 "Messages will be assigned to random partitions.");
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property. To specify partition, simply configure the 'partition' property.
+ */
 static final AllowableValue USER_DEFINED_PARTITIONING = new 
AllowableValue("User-Defined", "User-Defined",
 "The  property will be used to determine the partition. 
All messages within the same FlowFile will be "
 + "assigned to the same partition.");
@@ -120,19 +126,22 @@ public class PutKafka extends 
AbstractKafkaProcessor {
 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
 .expressionLanguageSupported(true)
 .build();
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property.
+ * This property is still valid as a dynamic property, so that existing 
processor configuration can stay valid.
+ */
 static final PropertyDescriptor PARTITION_STRATEGY = new 
PropertyDescriptor.Builder()
 .name("Partition Strategy")
-.description("Specifies how 

nifi git commit: NIFI-3363: PutKafka NPE with User-Defined partition

2017-01-27 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 3a0948188 -> 008bffd9c


NIFI-3363: PutKafka NPE with User-Defined partition

- Marked PutKafka Partition Strategy property as deprecated, as
  Kafka 0.8 client doesn't use 'partitioner.class' as producer property, we 
don't have to specify it.
- Changed Partition Strategy property from a required one to a dynamic 
property, so that existing processor config can stay in valid state.
- Fixed partition property to work.
- Route a flow file if it failed to be published due to invalid partition.


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/008bffd9
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/008bffd9
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/008bffd9

Branch: refs/heads/0.x
Commit: 008bffd9cd1787295840b411f1498439265bc8c5
Parents: 3a09481
Author: Koji Kawamura 
Authored: Wed Jan 18 17:44:40 2017 +0900
Committer: Oleg Zhurakousky 
Committed: Fri Jan 27 12:43:24 2017 -0500

--
 .../apache/nifi/processors/kafka/PutKafka.java  | 77 ++--
 .../nifi/processors/kafka/PutKafkaTest.java | 59 ++-
 2 files changed, 95 insertions(+), 41 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/008bffd9/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index abdf73d..38ec20c 100644
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -54,8 +54,7 @@ import 
org.apache.nifi.processors.kafka.KafkaPublisher.KafkaPublisherResult;
 
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub", "0.8.x"})
-@CapabilityDescription("Sends the contents of a FlowFile as a message to 
Apache Kafka, , specifically for 0.8.x versions. " +
-"The messages to send may be individual FlowFiles or may be delimited, 
using a "
+@CapabilityDescription("Sends the contents of a FlowFile as a message to 
Apache Kafka, specifically for 0.8.x versions. The messages to send may be 
individual FlowFiles or may be delimited, using a "
 + "user-specified delimiter, such as a new-line. The complementary 
NiFi processor for fetching messages is GetKafka.")
 @DynamicProperty(name = "The name of a Kafka configuration property.", value = 
"The value of a given Kafka configuration property.",
  description = "These properties will be added on the Kafka 
configuration after loading any provided configuration properties."
@@ -98,11 +97,20 @@ public class PutKafka extends 
AbstractKafkaProcessor {
 public static final AllowableValue COMPRESSION_CODEC_SNAPPY = new 
AllowableValue("snappy", "Snappy",
 "Compress messages using Snappy");
 
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property.
+ */
 static final AllowableValue ROUND_ROBIN_PARTITIONING = new 
AllowableValue("Round Robin", "Round Robin",
 "Messages will be assigned partitions in a round-robin fashion, 
sending the first message to Partition 1, "
 + "the next Partition to Partition 2, and so on, wrapping 
as necessary.");
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property.
+ */
 static final AllowableValue RANDOM_PARTITIONING = new 
AllowableValue("Random Robin", "Random",
 "Messages will be assigned to random partitions.");
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property. To specify partition, simply configure the 'partition' property.
+ */
 static final AllowableValue USER_DEFINED_PARTITIONING = new 
AllowableValue("User-Defined", "User-Defined",
 "The  property will be used to determine the partition. 
All messages within the same FlowFile will be "
 + "assigned to the same partition.");
@@ -121,19 +129,22 @@ public class PutKafka extends 
AbstractKafkaProcessor {
 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
 .expressionLanguageSupported(true)
 .build();
+/**
+ * @deprecated Kafka 0.8.x producer doesn't use 'partitioner.class' 
property.
+ * This property is still valid as a dynamic 

[1/2] nifi git commit: NIFI-2615 polishing - added missing POM entries to nifi POM and assembly POM - added available port-discovery to tests amongs other minor polishings

2017-01-25 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 1a6802a01 -> f0856565a


NIFI-2615 polishing
- added missing POM entries to nifi POM and assembly POM
- added available port-discovery to tests amongs other minor polishings


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f0856565
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f0856565
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f0856565

Branch: refs/heads/master
Commit: f0856565ad3979148c82fcef2865340a40ab61f5
Parents: 3626abd
Author: Oleg Zhurakousky 
Authored: Wed Jan 25 13:13:53 2017 -0500
Committer: Oleg Zhurakousky 
Committed: Wed Jan 25 13:22:50 2017 -0500

--
 nifi-assembly/pom.xml   |  5 ++
 .../apache/nifi/processors/gettcp/GetTCP.java   |  9 +--
 .../nifi/processors/gettcp/ReceivingClient.java | 25 +
 .../processors/gettcp/ReceivingClientTest.java  | 59 +++-
 .../apache/nifi/processors/gettcp/Server.java   |  2 +-
 .../nifi/processors/gettcp/TestGetTCP.java  | 44 +++
 nifi-nar-bundles/nifi-tcp-bundle/pom.xml|  1 -
 pom.xml |  6 ++
 8 files changed, 88 insertions(+), 63 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/f0856565/nifi-assembly/pom.xml
--
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index abe5a58..e015e03 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -418,6 +418,11 @@ language governing permissions and limitations under the 
License. -->
 nifi-websocket-processors-nar
 nar
 
+
+org.apache.nifi
+nifi-tcp-nar
+nar
+
 
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/f0856565/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
--
diff --git 
a/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
 
b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
index c366929..172a4f9 100644
--- 
a/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
+++ 
b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/GetTCP.java
@@ -40,6 +40,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessSessionFactory;
@@ -90,10 +91,10 @@ public class GetTCP extends AbstractSessionFactoryProcessor 
{
 public static final PropertyDescriptor RECEIVE_BUFFER_SIZE = new 
PropertyDescriptor.Builder()
 .name("receive-buffer-size")
 .displayName("Receive Buffer Size")
-.description("The size of the buffer to receive data in")
+.description("The size of the buffer to receive data in. Default 
16384 (16MB).")
 .required(false)
-.defaultValue("2048")
-.addValidator(StandardValidators.createLongValidator(1, 2048, 
true))
+.defaultValue("16MB")
+.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
 .build();
 
 public static final PropertyDescriptor END_OF_MESSAGE_BYTE = new 
PropertyDescriptor.Builder()
@@ -175,7 +176,7 @@ public class GetTCP extends AbstractSessionFactoryProcessor 
{
 
 @OnScheduled
 public void onScheduled(final ProcessContext context) throws 
ProcessException {
-this.receiveBufferSize = 
context.getProperty(RECEIVE_BUFFER_SIZE).asInteger();
+this.receiveBufferSize = 
context.getProperty(RECEIVE_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
 this.originalServerAddressList = 
context.getProperty(ENDPOINT_LIST).getValue();
 this.endOfMessageByte = ((byte) 
context.getProperty(END_OF_MESSAGE_BYTE).asInteger().intValue());
 this.connectionAttemptCount = 
context.getProperty(CONNECTION_ATTEMPT_COUNT).asInteger();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f0856565/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-processors/src/main/java/org/apache/nifi/processors/gettcp/ReceivingClient.java
--
diff --git 

[2/2] nifi git commit: NIFI-2615 Adding GetTCP processor

2017-01-25 Thread ozhurakousky
NIFI-2615 Adding GetTCP processor

NIFI-2615 Addressing changes from P/R. Specifically, removing .gitignore as it 
should not be there for a nar. Removed non-used class. Changed name in notice


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/3626abd7
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/3626abd7
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/3626abd7

Branch: refs/heads/master
Commit: 3626abd78a807a4ff806d51e7d27194f9fb7c961
Parents: 1a6802a
Author: Andrew Psaltis 
Authored: Thu Jan 19 18:28:21 2017 -0500
Committer: Oleg Zhurakousky 
Committed: Wed Jan 25 13:22:50 2017 -0500

--
 .../nifi-tcp-bundle/nifi-tcp-nar/pom.xml|  41 +++
 .../src/main/resources/META-INF/LICENSE | 212 +
 .../src/main/resources/META-INF/NOTICE  |  24 ++
 .../nifi-tcp-bundle/nifi-tcp-processors/pom.xml |  51 
 .../gettcp/AbstractSocketHandler.java   | 257 
 .../apache/nifi/processors/gettcp/GetTCP.java   | 302 +++
 .../nifi/processors/gettcp/GetTCPUtils.java |  85 ++
 .../nifi/processors/gettcp/MessageHandler.java  |  24 ++
 .../nifi/processors/gettcp/ReceivingClient.java | 166 ++
 .../org.apache.nifi.processor.Processor |  15 +
 .../processors/gettcp/ReceivingClientTest.java  | 199 
 .../apache/nifi/processors/gettcp/Server.java   |  91 ++
 .../nifi/processors/gettcp/TestGetTCP.java  | 118 
 .../src/test/resources/log4j.properties |   8 +
 nifi-nar-bundles/nifi-tcp-bundle/pom.xml|  35 +++
 nifi-nar-bundles/pom.xml|   1 +
 16 files changed, 1629 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/3626abd7/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/pom.xml 
b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/pom.xml
new file mode 100644
index 000..5227431
--- /dev/null
+++ b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/pom.xml
@@ -0,0 +1,41 @@
+
+
+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;>
+4.0.0
+
+
+org.apache.nifi
+nifi-tcp-bundle
+1.2.0-SNAPSHOT
+
+
+nifi-tcp-nar
+1.2.0-SNAPSHOT
+nar
+
+true
+true
+
+
+
+
+org.apache.nifi
+nifi-tcp-processors
+1.2.0-SNAPSHOT
+
+
+
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/3626abd7/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/src/main/resources/META-INF/LICENSE
--
diff --git 
a/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/src/main/resources/META-INF/LICENSE
 
b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 000..a57b09a
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-tcp-bundle/nifi-tcp-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,212 @@
+
+ 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 

nifi git commit: NIFI-3141: Fixed TailFile ArrayIndexOutOfBounds.

2016-12-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master afe742700 -> 8da38acf3


NIFI-3141: Fixed TailFile ArrayIndexOutOfBounds.

- Added unit test cases to simulate NiFi version update which fails without 
this fix.
- Added state object migration code, add file.0. prefix to state keys,
  and add length from stored position.

This closes #1289


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/8da38acf
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/8da38acf
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/8da38acf

Branch: refs/heads/master
Commit: 8da38acf31688569bcc0a1d79c2f90d2e4e535d4
Parents: afe7427
Author: Koji Kawamura 
Authored: Fri Dec 2 11:23:33 2016 +0900
Committer: Oleg Zhurakousky 
Committed: Fri Dec 2 07:53:04 2016 -0500

--
 .../nifi/processors/standard/TailFile.java  | 28 ++
 .../nifi/processors/standard/TestTailFile.java  | 94 
 2 files changed, 122 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/8da38acf/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
index 3553ce8..c5fcefb 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
@@ -345,6 +345,25 @@ public class TailFile extends AbstractProcessor {
 
 Map statesMap = stateMap.toMap();
 
+if (statesMap.containsKey(TailFileState.StateKeys.FILENAME)
+&& !statesMap.keySet().stream().anyMatch(key -> 
key.startsWith(MAP_PREFIX))) {
+// If statesMap contains "filename" key without "file.0." prefix,
+// and there's no key with "file." prefix, then
+// it indicates that the statesMap is created with earlier version 
of NiFi.
+// In this case, we need to migrate the state by adding prefix 
indexed with 0.
+final Map migratedStatesMap = new 
HashMap<>(statesMap.size());
+for (String key : statesMap.keySet()) {
+migratedStatesMap.put(MAP_PREFIX + "0." + key, 
statesMap.get(key));
+}
+
+// LENGTH is added from NiFi 1.1.0. Set the value with using the 
last position so that we can use existing state
+// to avoid sending duplicated log data after updating NiFi.
+migratedStatesMap.put(MAP_PREFIX + "0." + 
TailFileState.StateKeys.LENGTH, 
statesMap.get(TailFileState.StateKeys.POSITION));
+statesMap = Collections.unmodifiableMap(migratedStatesMap);
+
+getLogger().info("statesMap has been migrated. {}", new 
Object[]{migratedStatesMap});
+}
+
 initStates(filesToTail, statesMap, false);
 recoverState(context, filesToTail, statesMap);
 }
@@ -931,6 +950,15 @@ public class TailFile extends AbstractProcessor {
 Map updatedState = new HashMap();
 
 for(String key : oldState.toMap().keySet()) {
+// These states are stored by older version of NiFi, and won't 
be used anymore.
+// New states have 'file..' prefix.
+if (TailFileState.StateKeys.CHECKSUM.equals(key)
+|| TailFileState.StateKeys.FILENAME.equals(key)
+|| TailFileState.StateKeys.POSITION.equals(key)
+|| TailFileState.StateKeys.TIMESTAMP.equals(key)) {
+getLogger().info("Removed state {}={} stored by older 
version of NiFi.", new Object[]{key, oldState.get(key)});
+continue;
+}
 updatedState.put(key, oldState.get(key));
 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/8da38acf/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
index 

nifi git commit: NIFI-3066: Create unit test to reproduce bug that results in IllegalArgumentException: Cannot migrate FlowFiles from a Process Session to itself

2016-11-21 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master e731f0957 -> 2ee66de1a


NIFI-3066: Create unit test to reproduce bug that results in 
IllegalArgumentException: Cannot migrate FlowFiles from a Process Session to 
itself

NIFI-3066: Ensure that when a Bin is created, it is always created with its own 
new session

This closes #1245


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/2ee66de1
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/2ee66de1
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/2ee66de1

Branch: refs/heads/master
Commit: 2ee66de1a6f9afb495c6e6565f4d4ba9cff68f88
Parents: e731f09
Author: Mark Payne 
Authored: Fri Nov 18 15:21:56 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Mon Nov 21 11:34:14 2016 -0500

--
 .../nifi/processor/util/bin/BinFiles.java   |  2 +-
 .../processors/standard/TestMergeContent.java   | 26 
 2 files changed, 27 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/2ee66de1/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
--
diff --git 
a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
 
b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
index 2672491..67e37c2 100644
--- 
a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
+++ 
b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
@@ -278,7 +278,7 @@ public abstract class BinFiles extends 
AbstractSessionFactoryProcessor {
 for (final Map.Entry entry : 
flowFileGroups.entrySet()) {
 final Set unbinned = 
binManager.offer(entry.getKey(), entry.getValue(), session, sessionFactory);
 for (final FlowFile flowFile : unbinned) {
-Bin bin = new Bin(session, 0, Long.MAX_VALUE, 0, 
Integer.MAX_VALUE, null);
+Bin bin = new Bin(sessionFactory.createSession(), 0, 
Long.MAX_VALUE, 0, Integer.MAX_VALUE, null);
 bin.offer(flowFile, session);
 this.readyBins.add(bin);
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/2ee66de1/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
index 3a6d07c..6590d47 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestMergeContent.java
@@ -66,6 +66,32 @@ public class TestMergeContent {
 
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.standard",
 "DEBUG");
 }
 
+/**
+ * This test will verify that if we have a FlowFile larger than the Max 
Size for a Bin, it will go into its
+ * own bin and immediately be processed as its own bin.
+ */
+@Test
+public void testFlowFileLargerThanBin() {
+final TestRunner runner = TestRunners.newTestRunner(new 
MergeContent());
+runner.setProperty(MergeContent.MERGE_STRATEGY, 
MergeContent.MERGE_STRATEGY_BIN_PACK);
+runner.setProperty(MergeContent.MIN_ENTRIES, "2");
+runner.setProperty(MergeContent.MAX_ENTRIES, "2");
+runner.setProperty(MergeContent.MIN_SIZE, "1 KB");
+runner.setProperty(MergeContent.MAX_SIZE, "5 KB");
+
+runner.enqueue(new byte[1026]); // add flowfile that fits within the 
bin limits
+runner.enqueue(new byte[1024 * 6]); // add flowfile that is larger 
than the bin limit
+runner.run(2); // run twice so that we have a chance to create two 
bins (though we shouldn't create 2, because only 1 bin will be full)
+
+runner.assertTransferCount(MergeContent.REL_ORIGINAL, 1);
+runner.assertTransferCount(MergeContent.REL_MERGED, 1);
+runner.assertTransferCount(MergeContent.REL_FAILURE, 0);
+
+// Queue should not be empty because the first FlowFile will be 
transferred back to the input queue
+// when we run out @OnStopped logic, since it won't be transferred 

nifi git commit: NIFI-3020: - Introducing a strategy for identifying users. - Fixing issue with the referral strategy error message. - Adding code to shutdown the application when the authorizer or lo

2016-11-19 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 1be087147 -> c8830742e


NIFI-3020:
- Introducing a strategy for identifying users.
- Fixing issue with the referral strategy error message.
- Adding code to shutdown the application when the authorizer or login identity 
provider are not initialized successfully.

NIFI-3020:
- Updating the admin guide to document the identity strategy.

NIFI-3020:
- Ensuring the request replicator attempts to shutdown regardless of whether 
the flow service properly terminates.

This closes #1236


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c8830742
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c8830742
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c8830742

Branch: refs/heads/master
Commit: c8830742eee4470b371246ec9bf14d2fea5fa522
Parents: 1be0871
Author: Matt Gilman 
Authored: Wed Nov 16 16:29:14 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Sat Nov 19 08:38:41 2016 -0500

--
 .../src/main/asciidoc/administration-guide.adoc |  3 +
 .../resources/conf/login-identity-providers.xml |  7 ++-
 .../ApplicationStartupContextListener.java  | 58 
 .../org/apache/nifi/ldap/IdentityStrategy.java  | 26 +
 .../java/org/apache/nifi/ldap/LdapProvider.java | 36 ++--
 .../org/apache/nifi/ldap/ReferralStrategy.java  |  3 +-
 6 files changed, 102 insertions(+), 31 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c8830742/nifi-docs/src/main/asciidoc/administration-guide.adoc
--
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc 
b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 5322d86..14587f2 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -295,6 +295,7 @@ Below is an example and description of configuring a Login 
Identity Provider tha
 
 
 
+USE_DN
 12 hours
 
 
@@ -326,6 +327,8 @@ nifi.security.user.login.identity.provider=ldap-provider
 |`Url` | Url of the LDAP servier (i.e. ldap://:).
 |`User Search Base` | Base DN for searching for users (i.e. 
CN=Users,DC=example,DC=com).
 |`User Search Filter` | Filter for searching for users against the 'User 
Search Base'. (i.e. sAMAccountName={0}). The user specified name is inserted 
into '{0}'.
+|`Identity Strategy` | Strategy to identify users. Possible values are USE_DN 
and USE_USERNAME. The default functionality if this property is missing is 
USE_DN in order to retain backward
+compatibility. USE_DN will use the full DN of the user entry if possible. 
USE_USERNAME will use the username the user logged in with.
 |`Authentication Expiration` | The duration of how long the user 
authentication is valid for. If the user never logs out, they will be required 
to log back in following this duration.
 
|==
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/c8830742/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/login-identity-providers.xml
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/login-identity-providers.xml
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/login-identity-providers.xml
index a2beb4c..828868a 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/login-identity-providers.xml
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/login-identity-providers.xml
@@ -54,7 +54,11 @@
 'User Search Base' - Base DN for searching for users (i.e. 
CN=Users,DC=example,DC=com).
 'User Search Filter' - Filter for searching for users against the 
'User Search Base'.
 (i.e. sAMAccountName={0}). The user specified name is inserted 
into '{0}'.
-
+
+'Identity Strategy' - Strategy to identify users. Possible values are 
USE_DN and USE_USERNAME.
+The default functionality if this property is missing is USE_DN in 
order to retain
+backward compatibility. USE_DN will use the full DN of the user 
entry if possible.
+USE_USERNAME will use the username the user logged in with.
 'Authentication Expiration' - The duration of how long the user 
authentication is valid
 for. If the user never logs out, they will be required to log back 
in following
 

[6/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/1be08714/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
--
diff --git 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
index 2afaa70..324f59f 100644
--- 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
+++ 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/BufferedInputStream.java
@@ -16,19 +16,445 @@
  */
 package org.apache.nifi.stream.io;
 
+import java.io.IOException;
 import java.io.InputStream;
 
 /**
  * This class is a slight modification of the BufferedInputStream in the 
java.io package. The modification is that this implementation does not provide 
synchronization on method calls, which means
  * that this class is not suitable for use by multiple threads. However, the 
absence of these synchronized blocks results in potentially much better 
performance.
  */
-public class BufferedInputStream extends java.io.BufferedInputStream {
+public class BufferedInputStream extends InputStream {
 
-public BufferedInputStream(final InputStream in) {
-super(in);
+private final InputStream in;
+
+private static int DEFAULT_BUFFER_SIZE = 8192;
+
+/**
+ * The maximum size of array to allocate.
+ * Some VMs reserve some header words in an array.
+ * Attempts to allocate larger arrays may result in
+ * OutOfMemoryError: Requested array size exceeds VM limit
+ */
+private static int MAX_BUFFER_SIZE = Integer.MAX_VALUE - 8;
+
+/**
+ * The internal buffer array where the data is stored. When necessary,
+ * it may be replaced by another array of
+ * a different size.
+ */
+protected byte buf[];
+
+/**
+ * The index one greater than the index of the last valid byte in
+ * the buffer.
+ * This value is always
+ * in the range 0 through buf.length;
+ * elements buf[0] through buf[count-1]
+ * contain buffered input data obtained
+ * from the underlying input stream.
+ */
+private int count;
+
+/**
+ * The current position in the buffer. This is the index of the next
+ * character to be read from the buf array.
+ * 
+ * This value is always in the range 0
+ * through count. If it is less
+ * than count, then buf[pos]
+ * is the next byte to be supplied as input;
+ * if it is equal to count, then
+ * the next read or skip
+ * operation will require more bytes to be
+ * read from the contained input stream.
+ *
+ * @see java.io.BufferedInputStream#buf
+ */
+private int pos;
+
+/**
+ * The value of the pos field at the time the last
+ * mark method was called.
+ * 
+ * This value is always
+ * in the range -1 through pos.
+ * If there is no marked position in the input
+ * stream, this field is -1. If
+ * there is a marked position in the input
+ * stream, then buf[markpos]
+ * is the first byte to be supplied as input
+ * after a reset operation. If
+ * markpos is not -1,
+ * then all bytes from positions buf[markpos]
+ * through buf[pos-1] must remain
+ * in the buffer array (though they may be
+ * moved to another place in the buffer array,
+ * with suitable adjustments to the values
+ * of count, pos,
+ * and markpos); they may not
+ * be discarded unless and until the difference
+ * between pos and markpos
+ * exceeds marklimit.
+ *
+ * @see java.io.BufferedInputStream#mark(int)
+ * @see java.io.BufferedInputStream#pos
+ */
+protected int markpos = -1;
+
+/**
+ * The maximum read ahead allowed after a call to the
+ * mark method before subsequent calls to the
+ * reset method fail.
+ * Whenever the difference between pos
+ * and markpos exceeds marklimit,
+ * then the mark may be dropped by setting
+ * markpos to -1.
+ *
+ * @see java.io.BufferedInputStream#mark(int)
+ * @see java.io.BufferedInputStream#reset()
+ */
+protected int marklimit;
+
+/**
+ * Check to make sure that underlying input stream has not been
+ * nulled out due to close; if not return it;
+ */
+private InputStream getInIfOpen() throws IOException {
+InputStream input = in;
+if (input == null) {
+throw new IOException("Stream closed");
+}
+return input;
+}
+
+/**
+ * Check to make sure that buffer has not been nulled out due to
+ * close; if not return it;
+ */
+private byte[] getBufIfOpen() throws IOException {
+if (buf == null) {
+throw new IOException("Stream closed");
+}
+return buf;
+}
+
+/**
+ * Creates a BufferedInputStream
+

[1/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 5a25884f5 -> 1be087147


http://git-wip-us.apache.org/repos/asf/nifi/blob/1be08714/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java
new file mode 100644
index 000..bae2364
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java
@@ -0,0 +1,204 @@
+/*
+ * 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.provenance;
+
+import static org.apache.nifi.provenance.TestUtil.createFlowFile;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.serialization.RecordReader;
+import org.apache.nifi.provenance.serialization.RecordWriter;
+import org.apache.nifi.provenance.toc.StandardTocReader;
+import org.apache.nifi.provenance.toc.StandardTocWriter;
+import org.apache.nifi.provenance.toc.TocReader;
+import org.apache.nifi.provenance.toc.TocUtil;
+import org.apache.nifi.provenance.toc.TocWriter;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class AbstractTestRecordReaderWriter {
+@BeforeClass
+public static void setLogLevel() {
+
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance", 
"INFO");
+}
+
+protected ProvenanceEventRecord createEvent() {
+final Map attributes = new HashMap<>();
+attributes.put("filename", "1.txt");
+attributes.put("uuid", UUID.randomUUID().toString());
+
+final ProvenanceEventBuilder builder = new 
StandardProvenanceEventRecord.Builder();
+builder.setEventTime(System.currentTimeMillis());
+builder.setEventType(ProvenanceEventType.RECEIVE);
+builder.setTransitUri("nifi://unit-test");
+builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
+builder.setComponentId("1234");
+builder.setComponentType("dummy processor");
+final ProvenanceEventRecord record = builder.build();
+
+return record;
+}
+
+@Test
+public void testSimpleWriteWithToc() throws IOException {
+final File journalFile = new File("target/storage/" + 
UUID.randomUUID().toString() + "/testSimpleWrite");
+final File tocFile = TocUtil.getTocFile(journalFile);
+final TocWriter tocWriter = new StandardTocWriter(tocFile, false, 
false);
+final RecordWriter writer = createWriter(journalFile, tocWriter, 
false, 1024 * 1024);
+
+writer.writeHeader(1L);
+writer.writeRecord(createEvent(), 1L);
+writer.close();
+
+final TocReader tocReader = new StandardTocReader(tocFile);
+
+try (final FileInputStream fis = new FileInputStream(journalFile);
+final RecordReader reader = createReader(fis, 
journalFile.getName(), tocReader, 2048)) {
+assertEquals(0, reader.getBlockIndex());
+reader.skipToBlock(0);
+final StandardProvenanceEventRecord recovered = 
reader.nextRecord();
+assertNotNull(recovered);
+
+assertEquals("nifi://unit-test", recovered.getTransitUri());
+assertNull(reader.nextRecord());
+}
+
+FileUtils.deleteFile(journalFile.getParentFile(), true);
+}
+
+
+@Test
+public void 

[7/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
NIFI-2854: Refactor repositories and swap files to use schema-based 
serialization so that nifi can be rolled back to a previous version after an 
upgrade.

NIFI-2854: Incorporated PR review feedback

NIFI-2854: Implemented feedback from PR Review

NIFI-2854: Ensure that all resources are closed on 
CompressableRecordReader.close() even if an IOException is thrown when closing 
one of them

This closes #1202


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/1be08714
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/1be08714
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/1be08714

Branch: refs/heads/master
Commit: 1be08714731f01347ac1f98e18047fe7d9ab8afd
Parents: 5a25884
Author: Mark Payne 
Authored: Tue Oct 4 09:38:14 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Nov 18 14:53:13 2016 -0500

--
 .../nifi/provenance/NamedSearchableField.java   |   9 +-
 .../StandardProvenanceEventRecord.java  |  38 ++
 nifi-commons/nifi-schema-utils/pom.xml  |  26 +
 .../repository/schema/ComplexRecordField.java   |  95 
 .../nifi/repository/schema/FieldMapRecord.java  |  81 +++
 .../nifi/repository/schema/FieldType.java   |  74 +++
 .../nifi/repository/schema/MapRecordField.java  |  75 +++
 .../nifi/repository/schema/NamedValue.java  |  36 ++
 .../apache/nifi/repository/schema/Record.java   |  30 +
 .../nifi/repository/schema/RecordField.java |  30 +
 .../nifi/repository/schema/RecordSchema.java| 188 +++
 .../nifi/repository/schema/Repetition.java  |  22 +
 .../repository/schema/SchemaRecordReader.java   | 191 +++
 .../repository/schema/SchemaRecordWriter.java   | 139 +
 .../repository/schema/SimpleRecordField.java|  84 +++
 .../repository/schema/UnionRecordField.java |  64 +++
 .../schema/TestSchemaRecordReader.java  | 281 ++
 .../schema/TestSchemaRecordReaderWriter.java| 178 ++
 .../nifi/stream/io/BufferedInputStream.java | 436 ++-
 .../java/org/apache/nifi/util/FormatUtils.java  |  38 ++
 .../util/timebuffer/CountSizeEntityAccess.java  |  43 ++
 .../nifi/util/timebuffer/TimedCountSize.java|  41 ++
 .../org/wali/MinimalLockingWriteAheadLog.java   | 118 ++--
 .../src/main/java/org/wali/SerDe.java   |  30 +-
 .../src/main/java/org/wali/SerDeFactory.java|  60 ++
 .../java/org/wali/SingletonSerDeFactory.java|  46 ++
 nifi-commons/pom.xml|   1 +
 .../repository/claim/ResourceClaim.java |  24 +
 .../nifi-framework/nifi-framework-core/pom.xml  |   4 +
 .../nifi/controller/FileSystemSwapManager.java  | 416 ++
 .../nifi/controller/StandardFlowFileQueue.java  |   6 +
 .../repository/RepositoryRecordSerde.java   |  68 +++
 .../RepositoryRecordSerdeFactory.java   |  95 
 .../repository/SchemaRepositoryRecordSerde.java | 213 
 .../repository/StandardProcessSession.java  |  70 +--
 .../WriteAheadFlowFileRepository.java   | 547 +--
 .../WriteAheadRepositoryRecordSerde.java| 517 ++
 .../repository/claim/StandardResourceClaim.java |  23 -
 .../claim/StandardResourceClaimManager.java |   7 +-
 .../repository/io/ByteCountingInputStream.java  | 101 
 .../repository/io/ByteCountingOutputStream.java |  63 ---
 .../repository/schema/ContentClaimFieldMap.java |  92 
 .../repository/schema/ContentClaimSchema.java   |  63 +++
 .../schema/FlowFileRecordFieldMap.java  |  99 
 .../repository/schema/FlowFileSchema.java   |  67 +++
 .../schema/RepositoryRecordFieldMap.java|  83 +++
 .../schema/RepositoryRecordSchema.java  |  93 
 .../schema/RepositoryRecordUpdate.java  |  69 +++
 .../schema/ResourceClaimFieldMap.java   |  85 +++
 .../controller/swap/SchemaSwapDeserializer.java |  77 +++
 .../controller/swap/SchemaSwapSerializer.java   | 101 
 .../controller/swap/SimpleSwapDeserializer.java | 303 ++
 .../controller/swap/SimpleSwapSerializer.java   | 133 +
 .../nifi/controller/swap/SwapDeserializer.java  |  33 ++
 .../apache/nifi/controller/swap/SwapSchema.java |  79 +++
 .../nifi/controller/swap/SwapSerializer.java|  33 ++
 .../controller/swap/SwapSummaryFieldMap.java| 106 
 .../controller/TestFileSystemSwapManager.java   | 160 +-
 .../TestWriteAheadFlowFileRepository.java   |   2 +
 .../nifi/controller/swap/MockFlowFile.java  | 136 +
 .../TestSchemaSwapSerializerDeserializer.java   | 195 +++
 .../TestSimpleSwapSerializerDeserializer.java   | 139 +
 .../pom.xml |   4 +
 .../nifi/provenance/AbstractRecordWriter.java   | 173 ++
 .../provenance/ByteArraySchemaRecordReader.java |  80 +++
 .../provenance/ByteArraySchemaRecordWriter.java |  85 +++
 

[4/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/1be08714/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
new file mode 100644
index 000..76c208d
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
@@ -0,0 +1,92 @@
+/*
+ * 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.controller.repository.schema;
+
+import java.util.List;
+
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.controller.repository.claim.StandardContentClaim;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+
+public class ContentClaimFieldMap implements Record {
+private final ContentClaim contentClaim;
+private final long contentClaimOffset;
+private final ResourceClaimFieldMap resourceClaimFieldMap;
+private final RecordSchema schema;
+
+public ContentClaimFieldMap(final ContentClaim contentClaim, final long 
contentClaimOffset, final RecordSchema schema) {
+this.contentClaim = contentClaim;
+this.contentClaimOffset = contentClaimOffset;
+this.schema = schema;
+
+final List resourceClaimFields = 
schema.getField(ContentClaimSchema.RESOURCE_CLAIM).getSubFields();
+final RecordSchema resourceClaimSchema = new 
RecordSchema(resourceClaimFields);
+this.resourceClaimFieldMap = new 
ResourceClaimFieldMap(contentClaim.getResourceClaim(), resourceClaimSchema);
+}
+
+@Override
+public Object getFieldValue(final String fieldName) {
+switch (fieldName) {
+case ContentClaimSchema.RESOURCE_CLAIM:
+return resourceClaimFieldMap;
+case ContentClaimSchema.CONTENT_CLAIM_LENGTH:
+return contentClaim.getLength();
+case ContentClaimSchema.CONTENT_CLAIM_OFFSET:
+return contentClaimOffset;
+case ContentClaimSchema.RESOURCE_CLAIM_OFFSET:
+return contentClaim.getOffset();
+default:
+return null;
+}
+}
+
+@Override
+public RecordSchema getSchema() {
+return schema;
+}
+
+@Override
+public String toString() {
+return "ContentClaimFieldMap[" + contentClaim + "]";
+}
+
+public static ContentClaim getContentClaim(final Record claimRecord, final 
ResourceClaimManager resourceClaimManager) {
+final Record resourceClaimRecord = (Record) 
claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM);
+final String container = (String) 
resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
+final String section = (String) 
resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
+final String identifier = (String) 
resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
+final Boolean lossTolerant = (Boolean) 
resourceClaimRecord.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
+
+final Long length = (Long) 
claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_LENGTH);
+final Long resourceOffset = (Long) 
claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM_OFFSET);
+
+final ResourceClaim resourceClaim = 
resourceClaimManager.newResourceClaim(container, section, identifier, 
lossTolerant, false);
+final StandardContentClaim contentClaim = new 

[3/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/1be08714/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
index 97226b2..46bea31 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
@@ -22,26 +22,20 @@ import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.file.Files;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.repository.FlowFileRecord;
+import org.apache.nifi.controller.repository.FlowFileRepository;
 import org.apache.nifi.controller.repository.SwapContents;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.SwapManagerInitializationContext;
 import org.apache.nifi.controller.repository.claim.ResourceClaim;
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.events.EventReporter;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -56,7 +50,9 @@ public class TestFileSystemSwapManager {
 final FlowFileQueue flowFileQueue = 
Mockito.mock(FlowFileQueue.class);
 
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
 
-final SwapContents swapContents = 
FileSystemSwapManager.deserializeFlowFiles(in, 
"/src/test/resources/old-swap-file.swap", flowFileQueue, new 
NopResourceClaimManager());
+final FileSystemSwapManager swapManager = createSwapManager();
+final SwapContents swapContents = 
swapManager.peek("src/test/resources/old-swap-file.swap", flowFileQueue);
+
 final List records = swapContents.getFlowFiles();
 assertEquals(1, records.size());
 
@@ -67,53 +63,32 @@ public class TestFileSystemSwapManager {
 }
 }
 
-@Test
-public void testRoundTripSerializeDeserialize() throws IOException {
-final List toSwap = new ArrayList<>(1);
-final Map attrs = new HashMap<>();
-for (int i = 0; i < 1; i++) {
-attrs.put("i", String.valueOf(i));
-final FlowFileRecord ff = new TestFlowFile(attrs, i);
-toSwap.add(ff);
-}
-
-final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
-
Mockito.when(flowFileQueue.getIdentifier()).thenReturn("87bb99fe-412c-49f6-a441-d1b0af4e20b4");
 
-final String swapLocation = "target/testRoundTrip.swap";
-final File swapFile = new File(swapLocation);
-Files.deleteIfExists(swapFile.toPath());
+private FileSystemSwapManager createSwapManager() {
+final FileSystemSwapManager swapManager = new FileSystemSwapManager();
+final ResourceClaimManager resourceClaimManager = new 
NopResourceClaimManager();
+final FlowFileRepository flowfileRepo = 
Mockito.mock(FlowFileRepository.class);
+swapManager.initialize(new SwapManagerInitializationContext() {
+@Override
+public ResourceClaimManager getResourceClaimManager() {
+return resourceClaimManager;
+}
 
-try (final FileOutputStream fos = new FileOutputStream(swapFile)) {
-FileSystemSwapManager.serializeFlowFiles(toSwap, flowFileQueue, 
swapLocation, fos);
-}
+@Override
+public FlowFileRepository getFlowFileRepository() {
+return flowfileRepo;
+}
 
-final SwapContents swappedIn;
-try (final FileInputStream fis = new FileInputStream(swapFile);
-final DataInputStream dis = new DataInputStream(fis)) {
-swappedIn = FileSystemSwapManager.deserializeFlowFiles(dis, 
swapLocation, flowFileQueue, Mockito.mock(ResourceClaimManager.class));
-}
+@Override
+public EventReporter 

[2/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/1be08714/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java
index 35832c4..a95bd4f 100644
--- 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java
@@ -17,239 +17,142 @@
 package org.apache.nifi.provenance;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Collection;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.nifi.provenance.serialization.CompressableRecordWriter;
 import org.apache.nifi.provenance.serialization.RecordWriter;
 import org.apache.nifi.provenance.toc.TocWriter;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.ByteCountingOutputStream;
 import org.apache.nifi.stream.io.DataOutputStream;
-import org.apache.nifi.stream.io.GZIPOutputStream;
-import org.apache.nifi.stream.io.NonCloseableOutputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class StandardRecordWriter implements RecordWriter {
+/**
+ * @deprecated Deprecated in favor of SchemaRecordWriter
+ */
+@Deprecated
+public class StandardRecordWriter extends CompressableRecordWriter implements 
RecordWriter {
 private static final Logger logger = 
LoggerFactory.getLogger(StandardRecordWriter.class);
+public static final int SERIALIZATION_VERISON = 9;
+public static final String SERIALIZATION_NAME = 
"org.apache.nifi.provenance.PersistentProvenanceRepository";
 
 private final File file;
-private final FileOutputStream fos;
-private final ByteCountingOutputStream rawOutStream;
-private final TocWriter tocWriter;
-private final boolean compressed;
-private final int uncompressedBlockSize;
-private final AtomicBoolean dirtyFlag = new AtomicBoolean(false);
-
-private DataOutputStream out;
-private ByteCountingOutputStream byteCountingOut;
-private long lastBlockOffset = 0L;
-private int recordCount = 0;
-private volatile boolean closed = false;
-
-private final Lock lock = new ReentrantLock();
 
 
 public StandardRecordWriter(final File file, final TocWriter writer, final 
boolean compressed, final int uncompressedBlockSize) throws IOException {
+super(file, writer, compressed, uncompressedBlockSize);
 logger.trace("Creating Record Writer for {}", file.getName());
 
 this.file = file;
-this.compressed = compressed;
-this.fos = new FileOutputStream(file);
-rawOutStream = new ByteCountingOutputStream(fos);
-this.uncompressedBlockSize = uncompressedBlockSize;
+}
 
-this.tocWriter = writer;
+public StandardRecordWriter(final OutputStream out, final TocWriter 
tocWriter, final boolean compressed, final int uncompressedBlockSize) throws 
IOException {
+super(out, tocWriter, compressed, uncompressedBlockSize);
+this.file = null;
 }
 
 @Override
-public synchronized File getFile() {
-return file;
+protected String getSerializationName() {
+return SERIALIZATION_NAME;
 }
 
 @Override
-public synchronized void writeHeader(final long firstEventId) throws 
IOException {
-if (isDirty()) {
-throw new IOException("Cannot update Provenance Repository because 
this Record Writer has already failed to write to the Repository");
-}
-
-try {
-lastBlockOffset = rawOutStream.getBytesWritten();
-resetWriteStream(firstEventId);
-
-out.writeUTF(PersistentProvenanceRepository.class.getName());
-out.writeInt(PersistentProvenanceRepository.SERIALIZATION_VERSION);
-out.flush();
-} catch (final IOException ioe) {
-markDirty();
-throw ioe;
-}
+protected int getSerializationVersion() {
+return SERIALIZATION_VERISON;
 }
 
-
-/**
- * Resets the streams to prepare for a new block
- * @param eventId the first id that will be written to the new block
- * @throws IOException if unable to flush/close the current streams 

[5/7] nifi git commit: NIFI-2854: Refactor repositories and swap files to use schema-based serialization so that nifi can be rolled back to a previous version after an upgrade.

2016-11-18 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/1be08714/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
new file mode 100644
index 000..916fd76
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
@@ -0,0 +1,213 @@
+/*
+ * 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.controller.repository;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.controller.repository.schema.ContentClaimFieldMap;
+import org.apache.nifi.controller.repository.schema.ContentClaimSchema;
+import org.apache.nifi.controller.repository.schema.FlowFileSchema;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordFieldMap;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordSchema;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordUpdate;
+import org.apache.nifi.repository.schema.FieldType;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.apache.nifi.repository.schema.Repetition;
+import org.apache.nifi.repository.schema.SchemaRecordReader;
+import org.apache.nifi.repository.schema.SchemaRecordWriter;
+import org.apache.nifi.repository.schema.SimpleRecordField;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+
+public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde 
implements SerDe {
+private static final int MAX_ENCODING_VERSION = 1;
+
+private final RecordSchema writeSchema = 
RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1;
+private final RecordSchema contentClaimSchema = 
ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1;
+
+private final ResourceClaimManager resourceClaimManager;
+private volatile RecordSchema recoverySchema;
+
+public SchemaRepositoryRecordSerde(final ResourceClaimManager 
resourceClaimManager) {
+this.resourceClaimManager = resourceClaimManager;
+}
+
+@Override
+public void writeHeader(final DataOutputStream out) throws IOException {
+writeSchema.writeTo(out);
+}
+
+@Override
+public void serializeEdit(final RepositoryRecord previousRecordState, 
final RepositoryRecord newRecordState, final DataOutputStream out) throws 
IOException {
+serializeRecord(newRecordState, out);
+}
+
+@Override
+public void serializeRecord(final RepositoryRecord record, final 
DataOutputStream out) throws IOException {
+final RecordSchema schema;
+switch (record.getType()) {
+case CREATE:
+case UPDATE:
+schema = RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1;
+break;
+case CONTENTMISSING:
+case DELETE:
+schema = RepositoryRecordSchema.DELETE_SCHEMA_V1;
+break;
+case SWAP_IN:
+schema = RepositoryRecordSchema.SWAP_IN_SCHEMA_V1;
+break;
+case SWAP_OUT:
+schema = RepositoryRecordSchema.SWAP_OUT_SCHEMA_V1;
+break;
+default:
+throw new IllegalArgumentException("Received Repository Record 
with unknown Update Type: " + record.getType()); // won't happen.
+}
+
+final RepositoryRecordFieldMap fieldMap = new 
RepositoryRecordFieldMap(record, schema, contentClaimSchema);

nifi git commit: NIFI-3059 Adds --ignore-source option to the ZooKeeper Migrator to allow data read from a source zookeeper to be written back to the same zookeeper Added unit test to test the --ignor

2016-11-17 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master c4be80068 -> 06d7ecd32


NIFI-3059 Adds --ignore-source option to the ZooKeeper Migrator to allow data 
read from a source zookeeper to be written back to the same zookeeper
Added unit test to test the --ignore-source option

This closes #1242


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/06d7ecd3
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/06d7ecd3
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/06d7ecd3

Branch: refs/heads/master
Commit: 06d7ecd324ff8c352c512f3f4709f77786f46325
Parents: c4be800
Author: Jeff Storck 
Authored: Thu Nov 17 16:17:46 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Thu Nov 17 18:24:58 2016 -0500

--
 .../toolkit/zkmigrator/ZooKeeperMigrator.java   |  4 +--
 .../zkmigrator/ZooKeeperMigratorMain.java   | 12 ++---
 .../zkmigrator/ZooKeeperMigratorTest.groovy | 26 
 3 files changed, 37 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/06d7ecd3/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
--
diff --git 
a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
 
b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
index c2ad166..c15286e 100644
--- 
a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
+++ 
b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigrator.java
@@ -120,7 +120,7 @@ class ZooKeeperMigrator {
 }
 }
 
-void writeZooKeeper(InputStream zkData, AuthMode authMode, byte[] 
authData) throws IOException, ExecutionException, InterruptedException {
+void writeZooKeeper(InputStream zkData, AuthMode authMode, byte[] 
authData, boolean ignoreSource) throws IOException, ExecutionException, 
InterruptedException {
 ZooKeeper zooKeeper = getZooKeeper(zooKeeperEndpointConfig, authMode, 
authData);
 JsonReader jsonReader = new JsonReader(new BufferedReader(new 
InputStreamReader(zkData)));
 Gson gson = new GsonBuilder().create();
@@ -132,7 +132,7 @@ class ZooKeeperMigrator {
 LOGGER.info("Source data was obtained from ZooKeeper: {}", 
sourceZooKeeperEndpointConfig);
 
Preconditions.checkArgument(!Strings.isNullOrEmpty(sourceZooKeeperEndpointConfig.getConnectString())
 && !Strings.isNullOrEmpty(sourceZooKeeperEndpointConfig.getPath()),
 "Source ZooKeeper %s from %s is invalid", 
sourceZooKeeperEndpointConfig, zkData);
-
Preconditions.checkState(!zooKeeperEndpointConfig.equals(sourceZooKeeperEndpointConfig),
+Preconditions.checkArgument(
!(zooKeeperEndpointConfig.equals(sourceZooKeeperEndpointConfig) && 
!ignoreSource),
 "Source ZooKeeper config %s for the data provided can not be 
the same as the configured destination ZooKeeper config %s",
 sourceZooKeeperEndpointConfig, zooKeeperEndpointConfig);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/06d7ecd3/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
--
diff --git 
a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
 
b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
index c8488a1..012618e 100644
--- 
a/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
+++ 
b/nifi-toolkit/nifi-toolkit-zookeeper-migrator/src/main/java/org/apache/nifi/toolkit/zkmigrator/ZooKeeperMigratorMain.java
@@ -40,8 +40,8 @@ public class ZooKeeperMigratorMain {
 private static final String JAVA_HOME = "JAVA_HOME";
 private static final String NIFI_TOOLKIT_HOME = "NIFI_TOOLKIT_HOME";
 private static final String HEADER = System.lineSeparator() + "A tool for 
importing and exporting data from ZooKeeper." + System.lineSeparator() + 
System.lineSeparator();
-private static final String FOOTER = new 
StringBuilder(System.lineSeparator()).append("Java home: ")
-
.append(System.getenv(JAVA_HOME)).append(System.lineSeparator()).append("NiFi 
Toolkit home: ").append(System.getenv(NIFI_TOOLKIT_HOME)).toString();
+private static final String FOOTER = System.lineSeparator() + "Java home: 
" +
+   

nifi git commit: NIFI-2950 Adding support for whole number hex values and a fromRadix function

2016-11-17 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master e5eda6370 -> c4be80068


NIFI-2950 Adding support for whole number hex values and a fromRadix function

NIFI-2950 Fixing typo

This closes #1161


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c4be8006
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c4be8006
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c4be8006

Branch: refs/heads/master
Commit: c4be800688bf23a3bdea8def75b84c0f4ded243d
Parents: e5eda63
Author: jpercivall 
Authored: Wed Oct 26 16:34:14 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Nov 17 07:27:51 2016 -0500

--
 .../language/antlr/AttributeExpressionLexer.g   |  1 +
 .../language/antlr/AttributeExpressionParser.g  |  2 +-
 .../attribute/expression/language/Query.java|  6 ++
 .../evaluation/cast/DecimalCastEvaluator.java   |  8 ++-
 .../evaluation/cast/NumberCastEvaluator.java| 10 +++-
 .../cast/WholeNumberCastEvaluator.java  | 13 -
 .../functions/FromRadixEvaluator.java   | 58 
 .../language/evaluation/util/NumberParsing.java |  3 +-
 .../expression/language/TestQuery.java  | 19 ++-
 .../asciidoc/expression-language-guide.adoc | 32 +++
 10 files changed, 141 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c4be8006/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
--
diff --git 
a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
 
b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
index cf76808..9375579 100644
--- 
a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
+++ 
b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g
@@ -167,6 +167,7 @@ MINUS : 'minus';
 MULTIPLY : 'multiply';
 DIVIDE : 'divide';
 MATH : 'math';
+FROM_RADIX : 'fromRadix';
 TO_RADIX : 'toRadix';
 OR : 'or';
 AND : 'and';

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4be8006/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
--
diff --git 
a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
 
b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
index 5542fb0..eb50a28 100644
--- 
a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
+++ 
b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g
@@ -75,7 +75,7 @@ tokens {
 // functions that return Strings
 zeroArgString : (TO_UPPER | TO_LOWER | TRIM | TO_STRING | URL_ENCODE | 
URL_DECODE | BASE64_ENCODE | BASE64_DECODE | ESCAPE_JSON | ESCAPE_XML | 
ESCAPE_CSV | ESCAPE_HTML3 | ESCAPE_HTML4 | UNESCAPE_JSON | UNESCAPE_XML | 
UNESCAPE_CSV | UNESCAPE_HTML3 | UNESCAPE_HTML4 ) LPAREN! RPAREN!;
 oneArgString : ((SUBSTRING_BEFORE | SUBSTRING_BEFORE_LAST | SUBSTRING_AFTER | 
SUBSTRING_AFTER_LAST | REPLACE_NULL | REPLACE_EMPTY |
-   PREPEND | APPEND | FORMAT | STARTS_WITH | 
ENDS_WITH | CONTAINS | JOIN | JSON_PATH) LPAREN! anyArg RPAREN!) |
+   PREPEND | APPEND | FORMAT | STARTS_WITH | 
ENDS_WITH | CONTAINS | JOIN | JSON_PATH | FROM_RADIX) LPAREN! anyArg RPAREN!) |
   (TO_RADIX LPAREN! anyArg (COMMA! anyArg)? RPAREN!);
 twoArgString : ((REPLACE | REPLACE_FIRST | REPLACE_ALL) LPAREN! anyArg COMMA! 
anyArg RPAREN!) |
   (SUBSTRING LPAREN! anyArg (COMMA! anyArg)? RPAREN!);

http://git-wip-us.apache.org/repos/asf/nifi/blob/c4be8006/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
--
diff --git 
a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
 
b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
index 49c7518..fb48b0f 100644
--- 

nifi git commit: NIFI-1002: Added WebSocket support.

2016-11-16 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 2f9ec0324 -> 31ea50bce


NIFI-1002: Added WebSocket support.

- Added LICENSE and NOTICE files.
- Fixed UnitTest errors.

This closes #1231


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/31ea50bc
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/31ea50bc
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/31ea50bc

Branch: refs/heads/master
Commit: 31ea50bce9c2faa012a47f05218b74db99b5d197
Parents: 2f9ec03
Author: Koji Kawamura 
Authored: Wed Nov 16 13:44:01 2016 +0900
Committer: Oleg Zhurakousky 
Committed: Wed Nov 16 12:17:37 2016 -0500

--
 .../src/main/resources/META-INF/LICENSE | 203 +++
 .../src/main/resources/META-INF/NOTICE  |  24 +++
 .../src/main/resources/META-INF/LICENSE | 203 +++
 .../src/main/resources/META-INF/NOTICE  |  24 +++
 .../src/main/resources/META-INF/LICENSE | 203 +++
 .../src/main/resources/META-INF/NOTICE  |  29 +++
 .../nifi-websocket-services-jetty/pom.xml   |   4 -
 .../websocket/TestJettyWebSocketClient.java |   2 +-
 .../websocket/TestJettyWebSocketServer.java |   2 +-
 9 files changed, 688 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/31ea50bc/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors-nar/src/main/resources/META-INF/LICENSE
--
diff --git 
a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors-nar/src/main/resources/META-INF/LICENSE
 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 000..6b0b127
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-processors-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,203 @@
+
+ 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 

[1/3] nifi git commit: NIFI-1002: Added WebSocket support.

2016-11-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master b026f0beb -> 26a5881d2


http://git-wip-us.apache.org/repos/asf/nifi/blob/26a5881d/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/ControllerServiceTestContext.java
--
diff --git 
a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/ControllerServiceTestContext.java
 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/ControllerServiceTestContext.java
new file mode 100644
index 000..062d528
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/ControllerServiceTestContext.java
@@ -0,0 +1,68 @@
+/*
+ * 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.websocket;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.util.MockControllerServiceInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class ControllerServiceTestContext {
+
+private final ConfigurationContext configurationContext = 
mock(ConfigurationContext.class);
+private final ValidationContext validationContext = 
mock(ValidationContext.class);
+private MockControllerServiceInitializationContext initializationContext;
+
+public ControllerServiceTestContext(ControllerService controllerService, 
String id) {
+initializationContext = new 
MockControllerServiceInitializationContext(controllerService, id);
+doAnswer(invocation -> 
configurationContext.getProperty(invocation.getArgumentAt(0, 
PropertyDescriptor.class)))
+
.when(validationContext).getProperty(any(PropertyDescriptor.class));
+controllerService.getPropertyDescriptors().forEach(prop -> 
setDefaultValue(prop));
+}
+
+public MockControllerServiceInitializationContext 
getInitializationContext() {
+return initializationContext;
+}
+
+public ConfigurationContext getConfigurationContext() {
+return configurationContext;
+}
+
+public MockPropertyValue setDefaultValue(PropertyDescriptor 
propertyDescriptor) {
+return setCustomValue(propertyDescriptor, 
propertyDescriptor.getDefaultValue());
+}
+
+public MockPropertyValue setCustomValue(PropertyDescriptor 
propertyDescriptor, String value) {
+final MockPropertyValue propertyValue = new MockPropertyValue(value, 
initializationContext);
+when(configurationContext.getProperty(eq(propertyDescriptor)))
+.thenReturn(propertyValue);
+return propertyValue;
+}
+
+public ValidationContext getValidationContext() {
+return validationContext;
+}
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/26a5881d/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/TestJettyWebSocketClient.java
--
diff --git 
a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/TestJettyWebSocketClient.java
 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/TestJettyWebSocketClient.java
new file mode 100644
index 000..f71f4f2
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-jetty/src/test/java/org/apache/nifi/websocket/TestJettyWebSocketClient.java
@@ -0,0 +1,63 @@
+/*
+ * 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 

[2/3] nifi git commit: NIFI-1002: Added WebSocket support.

2016-11-15 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/26a5881d/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/pom.xml
--
diff --git 
a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/pom.xml 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/pom.xml
new file mode 100644
index 000..bf9951b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/pom.xml
@@ -0,0 +1,41 @@
+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;>
+   
+   
+   org.apache.nifi
+   nifi-websocket-bundle
+   1.1.0-SNAPSHOT
+   
+   4.0.0
+   nifi-websocket-services-api
+   jar
+   
+   
+   org.apache.nifi
+   nifi-ssl-context-service-api
+   
+   
+   org.apache.nifi
+   nifi-processor-utils
+   
+   
+   log4j
+   log4j
+   1.2.17
+   test
+   
+   
+   org.apache.nifi
+   nifi-mock
+   test
+   
+   
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/26a5881d/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/src/main/java/org/apache/nifi/websocket/AbstractWebSocketService.java
--
diff --git 
a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/src/main/java/org/apache/nifi/websocket/AbstractWebSocketService.java
 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/src/main/java/org/apache/nifi/websocket/AbstractWebSocketService.java
new file mode 100644
index 000..fac1e42
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/src/main/java/org/apache/nifi/websocket/AbstractWebSocketService.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.websocket;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processor.Processor;
+
+import java.io.IOException;
+
+public abstract class AbstractWebSocketService extends 
AbstractControllerService implements WebSocketService {
+
+final protected WebSocketMessageRouters routers = new 
WebSocketMessageRouters();
+
+@Override
+public void registerProcessor(final String endpointId, final Processor 
processor) throws WebSocketConfigurationException {
+routers.registerProcessor(endpointId, processor);
+}
+
+@Override
+public boolean isProcessorRegistered(final String endpointId, final 
Processor processor) {
+return routers.isProcessorRegistered(endpointId, processor);
+}
+
+@Override
+public void deregisterProcessor(final String endpointId, final Processor 
processor) throws WebSocketConfigurationException {
+routers.deregisterProcessor(endpointId, processor);
+}
+
+@Override
+public void sendMessage(final String endpointId, final String sessionId, 
final SendMessage sendMessage) throws IOException, 
WebSocketConfigurationException {
+routers.sendMessage(endpointId, sessionId, sendMessage);
+}
+
+@Override
+public void disconnect(final String endpointId, final String sessionId, 
final String reason) throws IOException, WebSocketConfigurationException {
+routers.disconnect(endpointId, sessionId, reason);
+}
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/26a5881d/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/src/main/java/org/apache/nifi/websocket/AbstractWebSocketSession.java
--
diff --git 
a/nifi-nar-bundles/nifi-websocket-bundle/nifi-websocket-services-api/src/main/java/org/apache/nifi/websocket/AbstractWebSocketSession.java
 

[3/3] nifi git commit: NIFI-1002: Added WebSocket support.

2016-11-15 Thread ozhurakousky
NIFI-1002: Added WebSocket support.

NIFI-1002: Added WebSocket support.

- Reflecting review comments
- Added displayName to peroperty descriptors

NIFI-1002: Added WebSocket support.

This closes #1184

- Reflecting review comments:
  - Removed unnecessary use of NarCloseable.withComponentNarLoader.
  - Removed intermediate on memory queue to make it simpler and more
robust. Received messages in WebSocket layer now will be transferred
to downstream relationships directly.


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/26a5881d
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/26a5881d
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/26a5881d

Branch: refs/heads/master
Commit: 26a5881d21d4187abd5ec8a47895bbdf7c914857
Parents: b026f0b
Author: Koji Kawamura 
Authored: Tue Oct 18 10:43:42 2016 +0900
Committer: Oleg Zhurakousky 
Committed: Tue Nov 15 19:34:59 2016 -0500

--
 nifi-assembly/pom.xml   |  15 ++
 .../nifi-websocket-processors-nar/pom.xml   |  46 
 .../nifi-websocket-processors/pom.xml   |  64 +
 .../AbstractWebSocketGatewayProcessor.java  | 231 
 .../processors/websocket/ConnectWebSocket.java  | 118 
 .../processors/websocket/ListenWebSocket.java   | 128 +
 .../nifi/processors/websocket/PutWebSocket.java | 239 
 .../websocket/WebSocketProcessorAttributes.java |  32 +++
 .../org.apache.nifi.processor.Processor |  17 ++
 .../websocket/TestConnectWebSocket.java | 124 +
 .../websocket/TestListenWebSocket.java  | 213 +++
 .../processors/websocket/TestPutWebSocket.java  | 270 +++
 .../nifi-websocket-services-api-nar/pom.xml |  45 
 .../nifi-websocket-services-api/pom.xml |  41 +++
 .../websocket/AbstractWebSocketService.java |  53 
 .../websocket/AbstractWebSocketSession.java |  32 +++
 .../nifi/websocket/BinaryMessageConsumer.java   |  21 ++
 .../nifi/websocket/ConnectedListener.java   |  24 ++
 .../apache/nifi/websocket/MessageSender.java|  25 ++
 .../org/apache/nifi/websocket/SendMessage.java  |  23 ++
 .../nifi/websocket/TextMessageConsumer.java |  21 ++
 .../nifi/websocket/WebSocketClientService.java  |  39 +++
 .../WebSocketConfigurationException.java|  24 ++
 .../websocket/WebSocketConnectedMessage.java|  23 ++
 .../apache/nifi/websocket/WebSocketMessage.java |  79 ++
 .../nifi/websocket/WebSocketMessageRouter.java  | 114 
 .../nifi/websocket/WebSocketMessageRouters.java |  75 ++
 .../nifi/websocket/WebSocketServerService.java  |  40 +++
 .../apache/nifi/websocket/WebSocketService.java |  50 
 .../apache/nifi/websocket/WebSocketSession.java |  26 ++
 .../nifi/websocket/WebSocketSessionInfo.java|  31 +++
 .../websocket/TestWebSocketMessageRouter.java   |  87 ++
 .../websocket/TestWebSocketMessageRouters.java  |  57 
 .../nifi-websocket-services-jetty-nar/pom.xml   |  45 
 .../nifi-websocket-services-jetty/pom.xml   |  55 
 .../jetty/AbstractJettyWebSocketService.java|  99 +++
 .../websocket/jetty/JettyWebSocketClient.java   | 165 
 .../websocket/jetty/JettyWebSocketServer.java   | 253 +
 .../websocket/jetty/JettyWebSocketSession.java  |  72 +
 .../jetty/RoutingWebSocketListener.java |  56 
 ...org.apache.nifi.controller.ControllerService |  16 ++
 .../websocket/ControllerServiceTestContext.java |  68 +
 .../websocket/TestJettyWebSocketClient.java |  63 +
 .../TestJettyWebSocketCommunication.java| 219 +++
 .../TestJettyWebSocketSecureCommunication.java  |  67 +
 .../websocket/TestJettyWebSocketServer.java |  51 
 .../example/WebSocketClientExample.java | 100 +++
 .../example/WebSocketServerExample.java | 195 ++
 .../src/test/resources/certs/localhost-ks.jks   | Bin 0 -> 2246 bytes
 .../src/test/resources/certs/localhost-ts.jks   | Bin 0 -> 1816 bytes
 .../src/test/resources/certs/localhost.crt  | Bin 0 -> 891 bytes
 nifi-nar-bundles/nifi-websocket-bundle/pom.xml  |  39 +++
 nifi-nar-bundles/pom.xml|   1 +
 pom.xml |  28 ++
 54 files changed, 4019 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/26a5881d/nifi-assembly/pom.xml
--
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index aa12f85..f05ab90 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -400,6 +400,21 @@ language governing permissions and limitations under the 
License. -->
 nifi-windows-event-log-nar
 

nifi git commit: NIFI-3040: Fixed bug where we were generating a RepositoryRecord with an 'UPDATE' type instead of a 'CREATE' time for Replay of FlowFiles. This caused FlowFile to have no attributes w

2016-11-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 2614b8354 -> 9f3bf121f


NIFI-3040: Fixed bug where we were generating a RepositoryRecord with an 
'UPDATE' type instead of a 'CREATE' time for Replay of FlowFiles. This caused 
FlowFile to have no attributes when restored from the FlowFile Repository.

This closes #1224


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/9f3bf121
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/9f3bf121
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/9f3bf121

Branch: refs/heads/master
Commit: 9f3bf121f989765e630da4c7c46e1100fb40a248
Parents: 2614b83
Author: Mark Payne 
Authored: Tue Nov 15 12:59:02 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Tue Nov 15 16:37:51 2016 -0500

--
 .../src/main/java/org/apache/nifi/controller/FlowController.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/9f3bf121/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index a5789e9..6927944 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -3859,7 +3859,8 @@ public class FlowController implements EventAccess, 
ControllerServiceProvider, R
 provenanceRepository.registerEvent(replayEvent);
 
 // Update the FlowFile Repository to indicate that we have added the 
FlowFile to the flow
-final StandardRepositoryRecord record = new 
StandardRepositoryRecord(queue, flowFileRecord);
+final StandardRepositoryRecord record = new 
StandardRepositoryRecord(queue);
+record.setWorking(flowFileRecord);
 record.setDestination(queue);
 
flowFileRepository.updateRepository(Collections.singleton(record));
 



nifi git commit: NIFI-3003 Upgrading hadoop.version to 2.7.3 and fixing TDE issue with PutHDFS, ensuring clean up of instance class loaders, and adding classpath resource property to all HDFS processo

2016-11-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 3c694b641 -> fe59b3415


NIFI-3003 Upgrading hadoop.version to 2.7.3 and fixing TDE issue with PutHDFS, 
ensuring clean up of instance class loaders, and adding classpath resource 
property to all HDFS processors

NIFI-3003 Addressing review feedback

NIFI-3003 added minor notice updates

This closes #1219


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/fe59b341
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/fe59b341
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/fe59b341

Branch: refs/heads/master
Commit: fe59b3415c0215a9a4eb01c32838c5dbb584c009
Parents: 3c694b6
Author: Bryan Bende 
Authored: Wed Nov 9 16:42:27 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Tue Nov 15 16:23:27 2016 -0500

--
 nifi-assembly/NOTICE|   8 +-
 .../init/ControllerServiceInitializer.java  |   3 +
 .../init/ProcessorInitializer.java  |   3 +
 .../init/ReportingTaskingInitializer.java   |   3 +
 .../nifi/fingerprint/FingerprintFactory.java|   7 +
 .../nifi/groups/StandardProcessGroup.java   |  19 +-
 .../org/apache/nifi/nar/ExtensionManager.java   |  10 +-
 .../apache/nifi/nar/InstanceClassLoader.java|  23 +-
 .../hadoop/AbstractHadoopProcessor.java | 135 ++---
 .../apache/nifi/processors/hadoop/PutHDFS.java  | 281 ++-
 .../src/main/resources/META-INF/NOTICE  |   8 +
 .../src/main/resources/META-INF/NOTICE  |  14 +-
 pom.xml |   8 +-
 13 files changed, 319 insertions(+), 203 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/fe59b341/nifi-assembly/NOTICE
--
diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index fd340b0..e920ff6 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -914,18 +914,14 @@ The following binary components are provided under the 
Apache Software License v
  Java Native Access
  Copyright 2015 Java Native Access
 
-(ASLv2) HTrace Core
+(ASLv2) Apache HTrace Core
   The following NOTICE information applies:
-In addition, this product includes software dependencies. See
-the accompanying LICENSE.txt for a listing of dependencies
-that are NOT Apache licensed (with pointers to their licensing)
+Copyright 2016 The Apache Software Foundation
 
 Apache HTrace includes an Apache Thrift connector to Zipkin. Zipkin
 is a distributed tracing system that is Apache 2.0 Licensed.
 Copyright 2012 Twitter, Inc.
 
-
-
 (ASLv2) Groovy (org.codehaus.groovy:groovy:jar:2.4.5 - 
http://www.groovy-lang.org)
   The following NOTICE information applies:
   Groovy Language

http://git-wip-us.apache.org/repos/asf/nifi/blob/fe59b341/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java
index 90c1e24..8bef0d3 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java
@@ -26,6 +26,7 @@ import 
org.apache.nifi.documentation.mock.MockControllerServiceInitializationCon
 import org.apache.nifi.documentation.mock.MockComponentLogger;
 import org.apache.nifi.documentation.util.ReflectionUtils;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.reporting.InitializationException;
 
@@ -53,6 +54,8 @@ public class ControllerServiceInitializer implements 
ConfigurableComponentInitia
 final ComponentLog logger = new MockComponentLogger();
 final MockConfigurationContext context = new 
MockConfigurationContext();
 
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, 
controllerService, logger, context);
+} finally {
+
ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier());
 }
 }
 }


nifi git commit: NIFI-3025: Bump nifi-spark-receiver's jackson version to match Spark 2.0.1

2016-11-14 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 13ea90912 -> 8d3177c38


NIFI-3025: Bump nifi-spark-receiver's jackson version to match Spark 2.0.1

This closes #1207


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/8d3177c3
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/8d3177c3
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/8d3177c3

Branch: refs/heads/master
Commit: 8d3177c38a4fed3b7040f03d17f28b1fc76525d8
Parents: 13ea909
Author: Randy Gelhausen 
Authored: Thu Nov 10 18:44:52 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Mon Nov 14 09:33:55 2016 -0500

--
 nifi-external/nifi-spark-receiver/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/8d3177c3/nifi-external/nifi-spark-receiver/pom.xml
--
diff --git a/nifi-external/nifi-spark-receiver/pom.xml 
b/nifi-external/nifi-spark-receiver/pom.xml
index 72bdb8c..dfa6c46 100644
--- a/nifi-external/nifi-spark-receiver/pom.xml
+++ b/nifi-external/nifi-spark-receiver/pom.xml
@@ -41,7 +41,7 @@
 
 com.fasterxml.jackson.core
 jackson-databind
-2.4.4
+2.6.5
 
 
 



nifi git commit: NIFI-2999: When Cluster Coordinator changes, purge any old heartbeats so that we don't disconnect a node due to very old heartbeats

2016-11-11 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master b9ef0fb84 -> b73ba7f8d


NIFI-2999: When Cluster Coordinator changes, purge any old heartbeats so that 
we don't disconnect a node due to very old heartbeats

This closes #1210


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/b73ba7f8
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/b73ba7f8
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/b73ba7f8

Branch: refs/heads/master
Commit: b73ba7f8d4f6319881c26b8faad121ceb12041ab
Parents: b9ef0fb
Author: Mark Payne 
Authored: Fri Nov 11 11:11:36 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Fri Nov 11 15:13:49 2016 -0500

--
 .../nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java | 5 +
 .../heartbeat/ClusterProtocolHeartbeatMonitor.java| 6 ++
 .../coordination/heartbeat/TestAbstractHeartbeatMonitor.java  | 5 +
 .../main/java/org/apache/nifi/controller/FlowController.java  | 7 +++
 4 files changed, 23 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/b73ba7f8/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
index 988ba75..6a0937d 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
@@ -54,6 +54,11 @@ public interface HeartbeatMonitor {
 void removeHeartbeat(NodeIdentifier nodeId);
 
 /**
+ * Clears all heartbeats that have been received
+ */
+void purgeHeartbeats();
+
+/**
  * @return the address that heartbeats should be sent to when this node is 
elected coordinator.
  */
 String getHeartbeatAddress();

http://git-wip-us.apache.org/repos/asf/nifi/blob/b73ba7f8/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
index 6a8e575..3e98368 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java
@@ -133,6 +133,12 @@ public class ClusterProtocolHeartbeatMonitor extends 
AbstractHeartbeatMonitor im
 }
 
 @Override
+public synchronized void purgeHeartbeats() {
+logger.debug("Purging old heartbeats");
+heartbeatMessages.clear();
+}
+
+@Override
 public ProtocolMessage handle(final ProtocolMessage msg) throws 
ProtocolException {
 switch (msg.getType()) {
 case HEARTBEAT:

http://git-wip-us.apache.org/repos/asf/nifi/blob/b73ba7f8/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java
index 690cda8..6610231 100644
--- 

nifi git commit: NIFI-3008 Enabled expression language on BASE_DIRECTORY and FILENAME properties in TailFile

2016-11-09 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 3e892c55e -> cc2fbcaac


NIFI-3008 Enabled expression language on BASE_DIRECTORY and FILENAME properties 
in TailFile

This closes #1190


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/cc2fbcaa
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/cc2fbcaa
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/cc2fbcaa

Branch: refs/heads/master
Commit: cc2fbcaac457ebbfeaada4a0fba84db784bfc9a4
Parents: 3e892c5
Author: Jeff Storck 
Authored: Tue Nov 8 22:02:03 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Wed Nov 9 12:11:11 2016 -0500

--
 .../nifi/processors/standard/TailFile.java  | 21 ++--
 .../nifi/processors/standard/TestTailFile.java  | 21 
 2 files changed, 32 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/cc2fbcaa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
index ba2c9b8..a171977 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
@@ -126,7 +126,7 @@ public class TailFile extends AbstractProcessor {
 .name("tail-base-directory")
 .displayName("Base directory")
 .description("Base directory used to look for files to tail. This 
property is required when using Multifile mode.")
-.expressionLanguageSupported(false)
+.expressionLanguageSupported(true)
 .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
 .required(false)
 .build();
@@ -148,8 +148,8 @@ public class TailFile extends AbstractProcessor {
 .description("Path of the file to tail in case of single file 
mode. If using multifile mode, regular expression to find files "
 + "to tail in the base directory. In case recursivity is 
set to true, the regular expression will be used to match the "
 + "path starting from the base directory (see additional 
details for examples).")
-.expressionLanguageSupported(false)
-.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+.expressionLanguageSupported(true)
+.addValidator(StandardValidators.createRegexValidator(0, 
Integer.MAX_VALUE, true))
 .required(true)
 .build();
 
@@ -267,7 +267,7 @@ public class TailFile extends AbstractProcessor {
 final List results = new 
ArrayList<>(super.customValidate(context));
 
 
if(context.getProperty(MODE).getValue().equals(MODE_MULTIFILE.getValue())) {
-String path = context.getProperty(BASE_DIRECTORY).getValue();
+String path = 
context.getProperty(BASE_DIRECTORY).evaluateAttributeExpressions().getValue();
 if(path == null) {
 results.add(new 
ValidationResult.Builder().subject(BASE_DIRECTORY.getName()).valid(false)
 .explanation("Base directory property cannot be empty 
in Multifile mode.").build());
@@ -291,8 +291,8 @@ public class TailFile extends AbstractProcessor {
 }
 } else {
 long max = context.getProperty(MAXIMUM_AGE).getValue() == null 
? Long.MAX_VALUE : 
context.getProperty(MAXIMUM_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
-List filesToTail = 
getFilesToTail(context.getProperty(BASE_DIRECTORY).getValue(),
-context.getProperty(FILENAME).getValue(),
+List filesToTail = 
getFilesToTail(context.getProperty(BASE_DIRECTORY).evaluateAttributeExpressions().getValue(),
+
context.getProperty(FILENAME).evaluateAttributeExpressions().getValue(),
 context.getProperty(RECURSIVE).asBoolean(),
 max);
 
@@ -322,12 +322,12 @@ public class TailFile extends AbstractProcessor {
 List filesToTail = new ArrayList();
 
 
if(context.getProperty(MODE).getValue().equals(MODE_MULTIFILE.getValue())) {
-
filesToTail.addAll(getFilesToTail(context.getProperty(BASE_DIRECTORY).getValue(),
-context.getProperty(FILENAME).getValue(),
+   

nifi git commit: NIFI-2998: Add validator to Avro Record Name in InferAvroSchema

2016-11-08 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 78020825e -> 63bda32a8


NIFI-2998: Add validator to Avro Record Name in InferAvroSchema

This closes #1187


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/63bda32a
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/63bda32a
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/63bda32a

Branch: refs/heads/master
Commit: 63bda32a8befe7f0da01a6c543462f6b2b2daced
Parents: 7802082
Author: Matt Burgess 
Authored: Tue Nov 8 11:56:14 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Tue Nov 8 14:14:40 2016 -0500

--
 .../nifi/processors/kite/InferAvroSchema.java   |  7 --
 .../processors/kite/TestInferAvroSchema.java| 24 
 2 files changed, 29 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/63bda32a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
--
diff --git 
a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
 
b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
index aad48ae..0edbd2b 100644
--- 
a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
+++ 
b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/InferAvroSchema.java
@@ -57,6 +57,7 @@ import java.util.Set;
 import java.util.HashSet;
 import java.util.Collections;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
 
 @Tags({"kite", "avro", "infer", "schema", "csv", "json"})
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
@@ -111,6 +112,7 @@ public class InferAvroSchema
 public static final String CSV_MIME_TYPE = "text/csv";
 public static final String AVRO_MIME_TYPE = "application/avro-binary";
 public static final String AVRO_FILE_EXTENSION = ".avro";
+public static final Pattern AVRO_RECORD_NAME_PATTERN = 
Pattern.compile("[A-Za-z_]+[A-Za-z0-9_.]*[^.]");
 
 public static final PropertyDescriptor SCHEMA_DESTINATION = new 
PropertyDescriptor.Builder()
 .name("Schema Output Destination")
@@ -202,10 +204,11 @@ public class InferAvroSchema
 
 public static final PropertyDescriptor RECORD_NAME = new 
PropertyDescriptor.Builder()
 .name("Avro Record Name")
-.description("Value to be placed in the Avro record schema 
\"name\" field.")
+.description("Value to be placed in the Avro record schema 
\"name\" field. The value must adhere to the Avro naming "
++ "rules for fullname. If Expression Language is present 
then the evaluated value must adhere to the Avro naming rules.")
 .required(true)
 .expressionLanguageSupported(true)
-.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+
.addValidator(StandardValidators.createRegexMatchingValidator(AVRO_RECORD_NAME_PATTERN))
 .build();
 
 public static final PropertyDescriptor CHARSET = new 
PropertyDescriptor.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/63bda32a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
--
diff --git 
a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
 
b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
index 171a64a..e1a9ef9 100644
--- 
a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
+++ 
b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestInferAvroSchema.java
@@ -64,6 +64,30 @@ public class TestInferAvroSchema {
 }
 
 @Test
+public void testRecordName() throws Exception {
+
+// Dot at the end is invalid
+runner.setProperty(InferAvroSchema.RECORD_NAME, 
"org.apache.nifi.contact.");
+runner.assertNotValid();
+// Dashes are invalid
+runner.setProperty(InferAvroSchema.RECORD_NAME, "avro-schema");
+runner.assertNotValid();
+// Name cannot start with a digit
+runner.setProperty(InferAvroSchema.RECORD_NAME, "1Record");
+runner.assertNotValid();
+// Name cannot start with a dot
+

nifi git commit: NIFI-2841 Refactoring logic in SplitAvro RecordSplitter to avoid making two calls in a row to reader.hasNext()

2016-11-08 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master d1d053725 -> 78020825e


NIFI-2841 Refactoring logic in SplitAvro RecordSplitter to avoid making two 
calls in a row to reader.hasNext()

This closes #1088


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/78020825
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/78020825
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/78020825

Branch: refs/heads/master
Commit: 78020825e9edfee1842e3d642ef762e5be1ab4bd
Parents: d1d0537
Author: Bryan Bende 
Authored: Thu Sep 29 13:49:27 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Tue Nov 8 13:50:42 2016 -0500

--
 .../main/java/org/apache/nifi/processors/avro/SplitAvro.java  | 7 +--
 1 file changed, 5 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/78020825/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java
--
diff --git 
a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java
 
b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java
index e3eb6ec..83964fa 100644
--- 
a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java
+++ 
b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java
@@ -267,7 +267,8 @@ public class SplitAvro extends AbstractProcessor {
 }
 
 // while records are left, start a new split by 
spawning a FlowFile
-while (reader.hasNext()) {
+final AtomicReference hasNextHolder = new 
AtomicReference(reader.hasNext());
+while (hasNextHolder.get()) {
 FlowFile childFlowFile = 
session.create(originalFlowFile);
 childFlowFile = session.write(childFlowFile, new 
OutputStreamCallback() {
 @Override
@@ -277,11 +278,13 @@ public class SplitAvro extends AbstractProcessor {
 
 // append to the current FlowFile 
until no more records, or splitSize is reached
 int recordCount = 0;
-while (reader.hasNext() && recordCount 
< splitSize) {
+while (hasNextHolder.get() && 
recordCount < splitSize) {
 
recordHolder.set(reader.next(recordHolder.get()));
 
splitWriter.write(recordHolder.get());
 recordCount++;
+
hasNextHolder.set(reader.hasNext());
 }
+
 splitWriter.flush();
 } finally {
 splitWriter.close();



[2/3] nifi git commit: NIFI-2909 Adding per-instance class loading capability through @RequiresInstanceClassLoading annotation NIFI-1712 Applying per-instance class loading to HBaseClientService to al

2016-11-08 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d05372/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index 21747cf..dc8056d 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -33,6 +33,7 @@ import org.apache.nifi.controller.ValidationContextFactory;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
@@ -50,27 +51,26 @@ public abstract class AbstractReportingTaskNode extends 
AbstractConfiguredCompon
 private volatile String comment;
 private volatile ScheduledState scheduledState = ScheduledState.STOPPED;
 
-protected final VariableRegistry variableRegistry;
-
 public AbstractReportingTaskNode(final ReportingTask reportingTask, final 
String id,
-final ControllerServiceProvider controllerServiceProvider, final 
ProcessScheduler processScheduler,
-final ValidationContextFactory validationContextFactory, final 
VariableRegistry variableRegistry) {
+ final ControllerServiceProvider 
controllerServiceProvider, final ProcessScheduler processScheduler,
+ final ValidationContextFactory 
validationContextFactory, final VariableRegistry variableRegistry,
+ final ComponentLog logger) {
 
 this(reportingTask, id, controllerServiceProvider, processScheduler, 
validationContextFactory,
-reportingTask.getClass().getSimpleName(), 
reportingTask.getClass().getCanonicalName(),variableRegistry);
+reportingTask.getClass().getSimpleName(), 
reportingTask.getClass().getCanonicalName(),variableRegistry, logger);
 }
 
 
 public AbstractReportingTaskNode(final ReportingTask reportingTask, final 
String id,
-final ControllerServiceProvider controllerServiceProvider, final 
ProcessScheduler processScheduler,
-final ValidationContextFactory validationContextFactory,
-final String componentType, final String componentCanonicalClass, 
VariableRegistry variableRegistry) {
+ final ControllerServiceProvider 
controllerServiceProvider, final ProcessScheduler processScheduler,
+ final ValidationContextFactory 
validationContextFactory,
+ final String componentType, final String 
componentCanonicalClass, final VariableRegistry variableRegistry,
+ final ComponentLog logger) {
 
-super(reportingTask, id, validationContextFactory, 
controllerServiceProvider, componentType, componentCanonicalClass);
+super(reportingTask, id, validationContextFactory, 
controllerServiceProvider, componentType, componentCanonicalClass, 
variableRegistry, logger);
 this.reportingTask = reportingTask;
 this.processScheduler = processScheduler;
 this.serviceLookup = controllerServiceProvider;
-this.variableRegistry = variableRegistry;
 }
 
 @Override
@@ -115,7 +115,7 @@ public abstract class AbstractReportingTaskNode extends 
AbstractConfiguredCompon
 
 @Override
 public ConfigurationContext getConfigurationContext() {
-return new StandardConfigurationContext(this, serviceLookup, 
getSchedulingPeriod(), variableRegistry);
+return new StandardConfigurationContext(this, serviceLookup, 
getSchedulingPeriod(), getVariableRegistry());
 }
 
 @Override
@@ -135,17 +135,6 @@ public abstract class AbstractReportingTaskNode extends 
AbstractConfiguredCompon
 this.scheduledState = state;
 }
 
-@Override
-public void setProperty(final String name, final String value) {
-super.setProperty(name, value);
-}
-
-@Override
-public boolean removeProperty(String name) {
-return super.removeProperty(name);
-}
-
-
 public boolean isDisabled() {
 return 

[1/3] nifi git commit: NIFI-2909 Adding per-instance class loading capability through @RequiresInstanceClassLoading annotation NIFI-1712 Applying per-instance class loading to HBaseClientService to al

2016-11-08 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 2f0d9a34f -> d1d053725


http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d05372/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/ModifiesClasspathProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/ModifiesClasspathProcessor.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/ModifiesClasspathProcessor.java
new file mode 100644
index 000..3bcbe0d
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/ModifiesClasspathProcessor.java
@@ -0,0 +1,50 @@
+/*
+ * 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.test.processors;
+
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.List;
+
+@RequiresInstanceClassLoading
+public class ModifiesClasspathProcessor extends AbstractProcessor {
+
+private List properties;
+
+public ModifiesClasspathProcessor() {
+
+}
+
+public ModifiesClasspathProcessor(List properties) {
+this.properties = properties;
+}
+
+@Override
+protected List getSupportedPropertyDescriptors() {
+return properties;
+}
+
+@Override
+public void onTrigger(ProcessContext context, ProcessSession session) 
throws ProcessException {
+}
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d05372/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 000..fca1c19
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+org.apache.nifi.test.processors.ModifiesClasspathProcessor
+org.apache.nifi.test.processors.ModifiesClasspathNoAnnotationProcessor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d05372/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestClasspathResources/resource1.txt
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestClasspathResources/resource1.txt
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestClasspathResources/resource1.txt
new 

[3/3] nifi git commit: NIFI-2909 Adding per-instance class loading capability through @RequiresInstanceClassLoading annotation NIFI-1712 Applying per-instance class loading to HBaseClientService to al

2016-11-08 Thread ozhurakousky
NIFI-2909 Adding per-instance class loading capability through 
@RequiresInstanceClassLoading annotation
NIFI-1712 Applying per-instance class loading to HBaseClientService to allow 
specifying Phoenix Client JAR

-Refactoring the ClassLoading so that every processor, controller service, and 
reporting task gets an InstanceClassLoader with a parent of the NAR 
ClassLoader, and only components with @RequiresInstanceClassLoading will make a 
copy of the NAR ClassLoader resources, and addressing some review feedback

This closes #1156


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d1d05372
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d1d05372
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d1d05372

Branch: refs/heads/master
Commit: d1d053725b72d91fbfca1a2e86691e9c9a1a3f2f
Parents: 2f0d9a3
Author: Bryan Bende 
Authored: Mon Oct 10 09:27:57 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Tue Nov 8 13:21:27 2016 -0500

--
 .../behavior/RequiresInstanceClassLoading.java  |  42 +++
 .../nifi/components/PropertyDescriptor.java |  28 ++
 .../org/apache/nifi/util/file/FileUtils.java|   1 +
 .../util/file/classloader/ClassLoaderUtils.java |  79 --
 .../file/classloader/TestClassLoaderUtils.java  |  41 +++
 .../src/main/asciidoc/developer-guide.adoc  |  52 
 .../init/ControllerServiceInitializer.java  |   9 +-
 .../init/ProcessorInitializer.java  |   8 +-
 .../init/ReportingTaskingInitializer.java   |   8 +-
 .../controller/AbstractConfiguredComponent.java | 209 +-
 .../nifi/controller/ConfiguredComponent.java|  21 +-
 .../apache/nifi/controller/ProcessorNode.java   |   9 +-
 .../apache/nifi/controller/FlowController.java  |  60 ++--
 .../controller/StandardFlowSynchronizer.java|  23 +-
 .../nifi/controller/StandardProcessorNode.java  |  39 +--
 .../reporting/AbstractReportingTaskNode.java|  33 +--
 .../reporting/StandardReportingTaskNode.java|  11 +-
 .../scheduling/EventDrivenSchedulingAgent.java  |   8 +-
 .../scheduling/StandardProcessScheduler.java|   6 +-
 .../service/ControllerServiceLoader.java|  15 +-
 .../service/StandardControllerServiceNode.java  |  36 +--
 .../StandardControllerServiceProvider.java  |  16 +-
 .../tasks/ContinuallyRunConnectableTask.java|   4 +-
 .../tasks/ContinuallyRunProcessorTask.java  |   2 +-
 .../controller/tasks/ReportingTaskWrapper.java  |   4 +-
 .../nifi/groups/StandardProcessGroup.java   |   8 +-
 .../controller/TestStandardProcessorNode.java   | 275 ++-
 .../scheduling/TestProcessorLifecycle.java  |  37 +--
 .../TestStandardProcessScheduler.java   |  18 +-
 .../TestStandardControllerServiceProvider.java  |  76 ++---
 .../service/util/TestControllerService.java |   2 +-
 .../ModifiesClasspathNoAnnotationProcessor.java |  50 
 .../processors/ModifiesClasspathProcessor.java  |  50 
 .../org.apache.nifi.processor.Processor |  16 ++
 .../TestClasspathResources/resource1.txt|  15 +
 .../TestClasspathResources/resource2.txt|  15 +
 .../TestClasspathResources/resource3.txt|  15 +
 .../src/test/resources/logback-test.xml |   7 +-
 .../org/apache/nifi/nar/ExtensionManager.java   |  82 ++
 .../apache/nifi/nar/InstanceClassLoader.java| 147 ++
 .../java/org/apache/nifi/nar/NarCloseable.java  |  20 +-
 .../nifi/web/controller/ControllerFacade.java   |   2 +-
 .../dao/impl/StandardControllerServiceDAO.java  |  10 +-
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  10 +-
 .../web/dao/impl/StandardReportingTaskDAO.java  |  10 +-
 .../nifi/controller/MonitorMemoryTest.java  |  14 +-
 .../apache/nifi/hbase/HBaseClientService.java   |   8 +
 .../nifi/hbase/HBase_1_1_2_ClientService.java   |   5 +-
 48 files changed, 1280 insertions(+), 376 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d05372/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
--
diff --git 
a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
 
b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
new file mode 100644
index 000..f7566a6
--- /dev/null
+++ 
b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ 

[1/2] nifi git commit: NIFI-1322 fixed breaking changes introduced in previous commit

2016-11-04 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 50010fb34 -> c13cfa6ea


NIFI-1322 fixed breaking changes introduced in previous commit

This closes #1181


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c13cfa6e
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c13cfa6e
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c13cfa6e

Branch: refs/heads/master
Commit: c13cfa6ea6009070db74d7dd9be9c66703d56942
Parents: a7d0641
Author: Oleg Zhurakousky 
Authored: Fri Nov 4 08:22:24 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Nov 4 08:23:10 2016 -0400

--
 .../apache/nifi/processors/hadoop/PutHDFS.java  | 38 +++-
 1 file changed, 21 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c13cfa6e/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
--
diff --git 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
index cb49d59..90b25e0 100644
--- 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
+++ 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
@@ -75,15 +75,19 @@ import java.util.concurrent.TimeUnit;
 @SeeAlso(GetHDFS.class)
 public class PutHDFS extends AbstractHadoopProcessor {
 
-public static final String REPLACE = "replace";
-public static final String IGNORE = "ignore";
-public static final String FAIL = "fail";
-public static final String APPEND = "append";
-
-public static final AllowableValue REPLACE_RESOLUTION = new 
AllowableValue(REPLACE, REPLACE, "Replaces the existing file if any.");
-public static final AllowableValue IGNORE_RESOLUTION = new 
AllowableValue(IGNORE, IGNORE, "Ignores the flow file and routes it to 
success.");
-public static final AllowableValue FAIL_RESOLUTION = new 
AllowableValue(FAIL, FAIL, "Penalizes the flow file and routes it to failure.");
-public static final AllowableValue APPEND_RESOLUTION = new 
AllowableValue(APPEND, APPEND, "Appends to the existing file if any, creates a 
new file otherwise.");
+public static final String REPLACE_RESOLUTION = "replace";
+public static final String IGNORE_RESOLUTION = "ignore";
+public static final String FAIL_RESOLUTION = "fail";
+public static final String APPEND_RESOLUTION = "append";
+
+public static final AllowableValue REPLACE_RESOLUTION_AV = new 
AllowableValue(REPLACE_RESOLUTION,
+REPLACE_RESOLUTION, "Replaces the existing file if any.");
+public static final AllowableValue IGNORE_RESOLUTION_AV = new 
AllowableValue(IGNORE_RESOLUTION, IGNORE_RESOLUTION,
+"Ignores the flow file and routes it to success.");
+public static final AllowableValue FAIL_RESOLUTION_AV = new 
AllowableValue(FAIL_RESOLUTION, FAIL_RESOLUTION,
+"Penalizes the flow file and routes it to failure.");
+public static final AllowableValue APPEND_RESOLUTION_AV = new 
AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
+"Appends to the existing file if any, creates a new file 
otherwise.");
 
 public static final String BUFFER_SIZE_KEY = "io.file.buffer.size";
 public static final int BUFFER_SIZE_DEFAULT = 4096;
@@ -108,8 +112,8 @@ public class PutHDFS extends AbstractHadoopProcessor {
 .name("Conflict Resolution Strategy")
 .description("Indicates what should happen when a file with the 
same name already exists in the output directory")
 .required(true)
-.defaultValue(FAIL_RESOLUTION.getValue())
-.allowableValues(REPLACE_RESOLUTION, IGNORE_RESOLUTION, 
FAIL_RESOLUTION, APPEND_RESOLUTION)
+.defaultValue(FAIL_RESOLUTION_AV.getValue())
+.allowableValues(REPLACE_RESOLUTION_AV, IGNORE_RESOLUTION_AV, 
FAIL_RESOLUTION_AV, APPEND_RESOLUTION_AV)
 .build();
 
 public static final PropertyDescriptor BLOCK_SIZE = new 
PropertyDescriptor.Builder()
@@ -258,18 +262,18 @@ public class PutHDFS extends AbstractHadoopProcessor {
 // If destination file already exists, resolve that based on 
processor configuration
 if (destinationExists) {
 switch (conflictResponse) {
-case REPLACE:
+case REPLACE_RESOLUTION:
 if (hdfs.delete(copyFile, false)) {
 getLogger().info("deleted {} in order 

[2/2] nifi git commit: NIFI-1322 - PutHDFS - allow file append resolution

2016-11-04 Thread ozhurakousky
NIFI-1322 - PutHDFS - allow file append resolution


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/a7d06412
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/a7d06412
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/a7d06412

Branch: refs/heads/master
Commit: a7d06412f8e4809157bf1074d659b5420546190f
Parents: 50010fb
Author: Pierre Villard 
Authored: Wed Nov 2 20:38:52 2016 +0100
Committer: Oleg Zhurakousky 
Committed: Fri Nov 4 08:23:10 2016 -0400

--
 .../apache/nifi/processors/hadoop/PutHDFS.java  | 62 
 1 file changed, 39 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/a7d06412/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
--
diff --git 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
index 3a0cb48..cb49d59 100644
--- 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
+++ 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
@@ -31,6 +31,7 @@ import 
org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
@@ -74,9 +75,15 @@ import java.util.concurrent.TimeUnit;
 @SeeAlso(GetHDFS.class)
 public class PutHDFS extends AbstractHadoopProcessor {
 
-public static final String REPLACE_RESOLUTION = "replace";
-public static final String IGNORE_RESOLUTION = "ignore";
-public static final String FAIL_RESOLUTION = "fail";
+public static final String REPLACE = "replace";
+public static final String IGNORE = "ignore";
+public static final String FAIL = "fail";
+public static final String APPEND = "append";
+
+public static final AllowableValue REPLACE_RESOLUTION = new 
AllowableValue(REPLACE, REPLACE, "Replaces the existing file if any.");
+public static final AllowableValue IGNORE_RESOLUTION = new 
AllowableValue(IGNORE, IGNORE, "Ignores the flow file and routes it to 
success.");
+public static final AllowableValue FAIL_RESOLUTION = new 
AllowableValue(FAIL, FAIL, "Penalizes the flow file and routes it to failure.");
+public static final AllowableValue APPEND_RESOLUTION = new 
AllowableValue(APPEND, APPEND, "Appends to the existing file if any, creates a 
new file otherwise.");
 
 public static final String BUFFER_SIZE_KEY = "io.file.buffer.size";
 public static final int BUFFER_SIZE_DEFAULT = 4096;
@@ -101,8 +108,8 @@ public class PutHDFS extends AbstractHadoopProcessor {
 .name("Conflict Resolution Strategy")
 .description("Indicates what should happen when a file with the 
same name already exists in the output directory")
 .required(true)
-.defaultValue(FAIL_RESOLUTION)
-.allowableValues(REPLACE_RESOLUTION, IGNORE_RESOLUTION, 
FAIL_RESOLUTION)
+.defaultValue(FAIL_RESOLUTION.getValue())
+.allowableValues(REPLACE_RESOLUTION, IGNORE_RESOLUTION, 
FAIL_RESOLUTION, APPEND_RESOLUTION)
 .build();
 
 public static final PropertyDescriptor BLOCK_SIZE = new 
PropertyDescriptor.Builder()
@@ -246,21 +253,23 @@ public class PutHDFS extends AbstractHadoopProcessor {
 changeOwner(context, hdfs, configuredRootDirPath);
 }
 
+final boolean destinationExists = hdfs.exists(copyFile);
+
 // If destination file already exists, resolve that based on 
processor configuration
-if (hdfs.exists(copyFile)) {
+if (destinationExists) {
 switch (conflictResponse) {
-case REPLACE_RESOLUTION:
+case REPLACE:
 if (hdfs.delete(copyFile, false)) {
 getLogger().info("deleted {} in order to replace 
with the contents of {}",
 new Object[]{copyFile, flowFile});
 }
 break;
-case IGNORE_RESOLUTION:
+case IGNORE:
 

nifi git commit: NIFI-2948 Fix DynamoDB Cached Credentials

2016-11-04 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 0841f14c1 -> 50010fb34


NIFI-2948 Fix DynamoDB Cached Credentials

This closes #1182


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/50010fb3
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/50010fb3
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/50010fb3

Branch: refs/heads/master
Commit: 50010fb34064d9cf622e265ed57c9ab0be39318e
Parents: 0841f14
Author: James Wing 
Authored: Thu Nov 3 20:24:19 2016 -0700
Committer: Oleg Zhurakousky 
Committed: Fri Nov 4 07:04:25 2016 -0400

--
 .../processors/aws/dynamodb/AbstractDynamoDBProcessor.java | 6 ++
 1 file changed, 6 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/50010fb3/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
 
b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
index de56e69..e1a31a9 100644
--- 
a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
+++ 
b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/dynamodb/AbstractDynamoDBProcessor.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang3.StringUtils;
+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;
@@ -335,4 +336,9 @@ public abstract class AbstractDynamoDBProcessor extends 
AbstractAWSCredentialsPr
 return isConsistent;
 
 }
+
+@OnStopped
+public void onStopped() {
+this.dynamoDB = null;
+}
 }



nifi git commit: NIFI-2980: Fixed doc for Elasticsearch URL property in ElasticsearchHttp processors

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 1abd017c3 -> 9e4473404


NIFI-2980: Fixed doc for Elasticsearch URL property in ElasticsearchHttp 
processors

This closes #1173


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/9e447340
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/9e447340
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/9e447340

Branch: refs/heads/master
Commit: 9e44734042cb7dfb5e17b2cf2137f8641be5bf16
Parents: 1abd017
Author: Matt Burgess 
Authored: Wed Nov 2 14:00:17 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 16:18:30 2016 -0400

--
 .../elasticsearch/AbstractElasticsearchHttpProcessor.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/9e447340/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
 
b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
index d477d0e..c4121b5 100644
--- 
a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
+++ 
b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
@@ -53,7 +53,7 @@ public abstract class AbstractElasticsearchHttpProcessor 
extends AbstractElastic
 public static final PropertyDescriptor ES_URL = new 
PropertyDescriptor.Builder()
 .name("elasticsearch-http-url")
 .displayName("Elasticsearch URL")
-.description("Elasticsearch URL which will be connected to, 
including scheme, host, port, path. The default port for the REST API is 9200.")
+.description("Elasticsearch URL which will be connected to, 
including scheme (http, e.g.), host, and port. The default port for the REST 
API is 9200.")
 .required(true)
 .addValidator(StandardValidators.URL_VALIDATOR)
 .build();



nifi git commit: NIFI-2963 FetchHDFS should support Compression Codec property

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master b5550ffcf -> 1abd017c3


NIFI-2963 FetchHDFS should support Compression Codec property

This closes #1166


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/1abd017c
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/1abd017c
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/1abd017c

Branch: refs/heads/master
Commit: 1abd017c353d2b45319e3a9e03281bba170c7d55
Parents: b5550ff
Author: Pierre Villard 
Authored: Fri Oct 28 12:35:37 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 16:15:09 2016 -0400

--
 .../nifi/processors/hadoop/FetchHDFS.java   | 43 +-
 .../nifi/processors/hadoop/TestFetchHDFS.java   | 60 
 2 files changed, 100 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/1abd017c/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
--
diff --git 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
index 3b1cce2..cd5f76c 100644
--- 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
+++ 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@ -16,9 +16,13 @@
  */
 package org.apache.nifi.processors.hadoop;
 
-import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -29,6 +33,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
@@ -38,6 +43,7 @@ import org.apache.nifi.util.StopWatch;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -83,6 +89,7 @@ public class FetchHDFS extends AbstractHadoopProcessor {
 protected List getSupportedPropertyDescriptors() {
 final List props = new ArrayList<>(properties);
 props.add(FILENAME);
+props.add(COMPRESSION_CODEC);
 return props;
 }
 
@@ -116,10 +123,38 @@ public class FetchHDFS extends AbstractHadoopProcessor {
 return;
 }
 
+InputStream stream = null;
+CompressionCodec codec = null;
+Configuration conf = getConfiguration();
+final CompressionCodecFactory compressionCodecFactory = new 
CompressionCodecFactory(conf);
+final CompressionType compressionType = 
CompressionType.valueOf(context.getProperty(COMPRESSION_CODEC).toString());
+final boolean inferCompressionCodec = compressionType == 
CompressionType.AUTOMATIC;
+
+if(inferCompressionCodec) {
+codec = compressionCodecFactory.getCodec(path);
+} else if (compressionType != CompressionType.NONE) {
+codec = getCompressionCodec(context, getConfiguration());
+}
+
 final URI uri = path.toUri();
 final StopWatch stopWatch = new StopWatch(true);
-try (final FSDataInputStream inStream = hdfs.open(path, 16384)) {
-flowFile = session.importFrom(inStream, flowFile);
+try {
+
+final String outputFilename;
+final String originalFilename = path.getName();
+stream = hdfs.open(path, 16384);
+
+// Check if compression codec is defined (inferred or otherwise)
+if (codec != null) {
+stream = codec.createInputStream(stream);
+outputFilename = StringUtils.removeEnd(originalFilename, 
codec.getDefaultExtension());
+} else {
+outputFilename = originalFilename;
+}
+

nifi git commit: NIFI-2834 Better Configs for Fetch and Timeout

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 769530bea -> b5550ffcf


NIFI-2834 Better Configs for Fetch and Timeout

NIFI-2834 Better Configs for Fetch and Timeout

NIFI-2834 Better Configs for Fetch and Timeout

NIFI-2834 ofSec to of Mils

This closes #1167


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/b5550ffc
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/b5550ffc
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/b5550ffc

Branch: refs/heads/master
Commit: b5550ffcf5d3b968551b91a41f146b18dcc0a9ec
Parents: 769530b
Author: Joe N 
Authored: Sat Oct 29 11:48:20 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 16:10:37 2016 -0400

--
 .../azure/eventhub/GetAzureEventHub.java| 33 +++-
 .../azure/eventhub/GetAzureEventHubTest.java|  4 +++
 2 files changed, 36 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/b5550ffc/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
--
diff --git 
a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
 
b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
index 0455fe9..12ea1ba 100644
--- 
a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
+++ 
b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/eventhub/GetAzureEventHub.java
@@ -40,6 +40,7 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.util.StopWatch;
 
 import java.io.IOException;
+import java.time.Duration;
 import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -121,6 +122,20 @@ public class GetAzureEventHub extends AbstractProcessor {
 .expressionLanguageSupported(false)
 .required(false)
 .build();
+static final PropertyDescriptor RECEIVER_FETCH_SIZE = new 
PropertyDescriptor.Builder()
+.name("Partition Recivier Fetch Size")
+.description("The number of events that a receiver should fetch 
from an EventHubs partition before returning. Default(100)")
+.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+.expressionLanguageSupported(false)
+.required(false)
+.build();
+static final PropertyDescriptor RECEIVER_FETCH_TIMEOUT = new 
PropertyDescriptor.Builder()
+.name("Partiton Receiver Timeout (millseconds)")
+.description("The amount of time a Partition Receiver should wait 
to receive the Fetch Size before returning. Default(6)")
+.addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+.expressionLanguageSupported(false)
+.required(false)
+.build();
 
 static final Relationship REL_SUCCESS = new Relationship.Builder()
 .name("success")
@@ -130,6 +145,8 @@ public class GetAzureEventHub extends AbstractProcessor {
 private final ConcurrentMap 
partitionToReceiverMap = new ConcurrentHashMap<>();
 private volatile BlockingQueue partitionNames = new 
LinkedBlockingQueue<>();
 private volatile Instant configuredEnqueueTime;
+private volatile int receiverFetchSize;
+private volatile Duration receiverFetchTimeout;
 private EventHubClient eventHubClient;
 
 private final static List propertyDescriptors;
@@ -148,6 +165,9 @@ public class GetAzureEventHub extends AbstractProcessor {
 _propertyDescriptors.add(NUM_PARTITIONS);
 _propertyDescriptors.add(CONSUMER_GROUP);
 _propertyDescriptors.add(ENQUEUE_TIME);
+_propertyDescriptors.add(RECEIVER_FETCH_SIZE);
+_propertyDescriptors.add(RECEIVER_FETCH_TIMEOUT);
+
 propertyDescriptors = 
Collections.unmodifiableList(_propertyDescriptors);
 
 Set _relationships = new HashSet<>();
@@ -201,6 +221,7 @@ public class GetAzureEventHub extends AbstractProcessor {
 partitionId,
 configuredEnqueueTime == null ? Instant.now() : 
configuredEnqueueTime).get();
 
+receiver.setReceiveTimeout(receiverFetchTimeout == null ? 
Duration.ofMillis(6) : receiverFetchTimeout);
 partitionToReceiverMap.put(partitionId, receiver);
 return receiver;
 
@@ -222,7 +243,7 @@ public class GetAzureEventHub extends AbstractProcessor {
 final 

nifi git commit: NIFI-2918 JDBC getColumnName may return empty string

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 892c74dff -> 2fb3b01eb


NIFI-2918 JDBC getColumnName may return empty string

This closes #1149


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/2fb3b01e
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/2fb3b01e
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/2fb3b01e

Branch: refs/heads/master
Commit: 2fb3b01ebe2a0519b98e4450cd89204b4225657d
Parents: 892c74d
Author: patricker 
Authored: Thu Oct 20 09:56:22 2016 -0600
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 13:13:34 2016 -0400

--
 .../processors/standard/util/JdbcCommon.java|  3 ++-
 .../standard/util/TestJdbcCommon.java   | 25 
 2 files changed, 27 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/2fb3b01e/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
index 0aa4c60..f19f7e0 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
@@ -258,7 +258,8 @@ public class JdbcCommon {
  * Some missing Avro types - Decimal, Date types. May need some 
additional work.
  */
 for (int i = 1; i <= nrOfColumns; i++) {
-String columnName = convertNames ? 
normalizeNameForAvro(meta.getColumnName(i)) : meta.getColumnName(i);
+String nameOrLabel = StringUtils.isNotEmpty(meta.getColumnName(i)) 
? meta.getColumnName(i) : meta.getColumnLabel(i);
+String columnName = convertNames ? 
normalizeNameForAvro(nameOrLabel) : nameOrLabel;
 switch (meta.getColumnType(i)) {
 case CHAR:
 case LONGNVARCHAR:

http://git-wip-us.apache.org/repos/asf/nifi/blob/2fb3b01e/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
index dd375aa..1e0fe2f 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
@@ -151,6 +151,31 @@ public class TestJdbcCommon {
 }
 
 @Test
+public void testCreateSchemaOnlyColumnLabel() throws 
ClassNotFoundException, SQLException {
+
+final ResultSet resultSet = mock(ResultSet.class);
+final ResultSetMetaData resultSetMetaData = 
mock(ResultSetMetaData.class);
+when(resultSet.getMetaData()).thenReturn(resultSetMetaData);
+when(resultSetMetaData.getColumnCount()).thenReturn(2);
+when(resultSetMetaData.getTableName(1)).thenReturn("TEST");
+when(resultSetMetaData.getColumnType(1)).thenReturn(Types.INTEGER);
+when(resultSetMetaData.getColumnName(1)).thenReturn("");
+when(resultSetMetaData.getColumnLabel(1)).thenReturn("ID");
+when(resultSetMetaData.getColumnType(2)).thenReturn(Types.VARCHAR);
+when(resultSetMetaData.getColumnName(2)).thenReturn("VCHARC");
+when(resultSetMetaData.getColumnLabel(2)).thenReturn("NOT_VCHARC");
+
+final Schema schema = JdbcCommon.createSchema(resultSet);
+assertNotNull(schema);
+
+assertNotNull(schema.getField("ID"));
+assertNotNull(schema.getField("VCHARC"));
+
+// records name, should be result set first column table name
+assertEquals("TEST", schema.getName());
+}
+
+@Test
 public void testConvertToBytes() throws ClassNotFoundException, 
SQLException, IOException {
 final Statement st = con.createStatement();
 st.executeUpdate("insert into restaurants values (1, 'Irifunes', 'San 
Mateo')");



nifi git commit: NIFI-2819: Added support for Expresssion Language in ModifyBytes

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 05ea76dd6 -> 892c74dff


NIFI-2819: Added support for Expresssion Language in ModifyBytes

This closes #1130


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/892c74df
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/892c74df
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/892c74df

Branch: refs/heads/master
Commit: 892c74dff26c1b56860131489f6ee56018631c9d
Parents: 05ea76d
Author: Matt Burgess 
Authored: Thu Oct 13 09:50:56 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 13:02:25 2016 -0400

--
 .../nifi/processors/standard/ModifyBytes.java   |  6 ++--
 .../processors/standard/TestModifyBytes.java| 35 
 2 files changed, 39 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/892c74df/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
index 444a172..6d8bee3 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java
@@ -66,6 +66,7 @@ public class ModifyBytes extends AbstractProcessor {
 .required(true)
 .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
 .defaultValue("0 B")
+.expressionLanguageSupported(true)
 .build();
 public static final PropertyDescriptor END_OFFSET = new 
PropertyDescriptor.Builder()
 .name("End Offset")
@@ -73,6 +74,7 @@ public class ModifyBytes extends AbstractProcessor {
 .required(true)
 .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
 .defaultValue("0 B")
+.expressionLanguageSupported(true)
 .build();
 public static final PropertyDescriptor REMOVE_ALL = new 
PropertyDescriptor.Builder()
 .name("Remove All Content")
@@ -114,8 +116,8 @@ public class ModifyBytes extends AbstractProcessor {
 
 final ComponentLog logger = getLogger();
 
-final long startOffset = 
context.getProperty(START_OFFSET).asDataSize(DataUnit.B).longValue();
-final long endOffset = 
context.getProperty(END_OFFSET).asDataSize(DataUnit.B).longValue();
+final long startOffset = 
context.getProperty(START_OFFSET).evaluateAttributeExpressions(ff).asDataSize(DataUnit.B).longValue();
+final long endOffset = 
context.getProperty(END_OFFSET).evaluateAttributeExpressions(ff).asDataSize(DataUnit.B).longValue();
 final boolean removeAll = context.getProperty(REMOVE_ALL).asBoolean();
 final long newFileSize = removeAll ? 0L : ff.getSize() - startOffset - 
endOffset;
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/892c74df/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestModifyBytes.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestModifyBytes.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestModifyBytes.java
index 7243be8..91f0853 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestModifyBytes.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestModifyBytes.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.HashMap;
 
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -95,6 +96,24 @@ public class TestModifyBytes {
 }
 
 @Test
+public void testRemoveHeaderEL() throws IOException {
+final TestRunner runner = TestRunners.newTestRunner(new ModifyBytes());
+runner.setProperty(ModifyBytes.START_OFFSET, "${numBytes}"); //REMOVE 
- '<<>>'
+runner.setProperty(ModifyBytes.END_OFFSET, "0 MB");
+
+runner.enqueue(testFilePath, new HashMap() {{
+

nifi git commit: NIFI-2883 Renamed directories holding ConsumeKafka_0_10 and PublishKafka_0_10 additional details

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 7fbc23639 -> 5f9cae97b


NIFI-2883 Renamed directories holding ConsumeKafka_0_10 and PublishKafka_0_10 
additional details

This closes #1170


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/5f9cae97
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/5f9cae97
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/5f9cae97

Branch: refs/heads/master
Commit: 5f9cae97b763186e23b0c970ea4f3d9a8866a885
Parents: 7fbc236
Author: Andrew Lim 
Authored: Mon Oct 31 15:22:12 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 12:29:09 2016 -0400

--
 .../additionalDetails.html  | 33 --
 .../additionalDetails.html  | 33 ++
 .../additionalDetails.html  | 47 
 .../additionalDetails.html  | 47 
 4 files changed, 80 insertions(+), 80 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/5f9cae97/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html
deleted file mode 100644
index 204034e..000
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka/additionalDetails.html
+++ /dev/null
@@ -1,33 +0,0 @@
-
-
-
-
-
-ConsumeKafka
-
-
-
-
-
-Description:
-
-This Processor polls http://kafka.apache.org/;>Apache 
Kafka
-for data using KafkaConsumer API available with Kafka 0.10.x. When 
a message is received 
-from Kafka, this Processor emits a FlowFile where the content of 
the FlowFile is the value 
-of the Kafka message.
-
-
-

http://git-wip-us.apache.org/repos/asf/nifi/blob/5f9cae97/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10/additionalDetails.html
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10/additionalDetails.html
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10/additionalDetails.html
new file mode 100644
index 000..204034e
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafka_0_10/additionalDetails.html
@@ -0,0 +1,33 @@
+
+
+
+
+
+ConsumeKafka
+
+
+
+
+
+Description:
+
+This Processor polls http://kafka.apache.org/;>Apache 
Kafka
+for data using KafkaConsumer API available with Kafka 0.10.x. When 
a message is received 
+from Kafka, this Processor emits a FlowFile where the content of 
the FlowFile is the value 
+of the Kafka message.
+
+
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/5f9cae97/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html
deleted file mode 100644
index 9ebf874..000
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafka/additionalDetails.html
+++ /dev/null
@@ -1,47 +0,0 @@
-
-
-
-
-
-PublishKafka
-
-
-
-
-
-Description:
-
-This Processor puts the contents of a FlowFile to a Topic in
-

nifi git commit: NIFI-2956 - GetHDFS - fixed directly path evaluation

2016-11-02 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 4acc9ad28 -> 7fbc23639


NIFI-2956 - GetHDFS - fixed directly path evaluation


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/7fbc2363
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/7fbc2363
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/7fbc2363

Branch: refs/heads/master
Commit: 7fbc23639af654fcee72e8ef74340b9f093658d6
Parents: 4acc9ad
Author: Pierre Villard 
Authored: Thu Oct 27 16:37:29 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Wed Nov 2 12:15:26 2016 -0400

--
 .../apache/nifi/processors/hadoop/GetHDFS.java  | 46 
 1 file changed, 19 insertions(+), 27 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/7fbc2363/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
--
diff --git 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
index 7ab7ebe..24de0c4 100644
--- 
a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
+++ 
b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
@@ -415,7 +415,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
 try {
 final FileSystem hdfs = getFileSystem();
 // get listing
-listing = selectFiles(hdfs, 
processorConfig.getConfiguredRootDirPath(), null);
+listing = selectFiles(hdfs, new 
Path(context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue()), 
null);
 lastPollTime.set(System.currentTimeMillis());
 } finally {
 listingLock.unlock();
@@ -460,7 +460,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
 if (file.isDirectory() && processorConfig.getRecurseSubdirs()) {
 files.addAll(selectFiles(hdfs, canonicalFile, filesVisited));
 
-} else if (!file.isDirectory() && 
processorConfig.getPathFilter().accept(canonicalFile)) {
+} else if (!file.isDirectory() && 
processorConfig.getPathFilter(dir).accept(canonicalFile)) {
 final long fileAge = System.currentTimeMillis() - 
file.getModificationTime();
 if (processorConfig.getMinimumAge() < fileAge && fileAge < 
processorConfig.getMaximumAge()) {
 files.add(canonicalFile);
@@ -480,17 +480,14 @@ public class GetHDFS extends AbstractHadoopProcessor {
  */
 protected static class ProcessorConfiguration {
 
-final private Path configuredRootDirPath;
 final private Pattern fileFilterPattern;
 final private boolean ignoreDottedFiles;
 final private boolean filterMatchBasenameOnly;
 final private long minimumAge;
 final private long maximumAge;
 final private boolean recurseSubdirs;
-final private PathFilter pathFilter;
 
 ProcessorConfiguration(final ProcessContext context) {
-configuredRootDirPath = new 
Path(context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue());
 ignoreDottedFiles = 
context.getProperty(IGNORE_DOTTED_FILES).asBoolean();
 final String fileFilterRegex = 
context.getProperty(FILE_FILTER_REGEX).getValue();
 fileFilterPattern = (fileFilterRegex == null) ? null : 
Pattern.compile(fileFilterRegex);
@@ -500,7 +497,22 @@ public class GetHDFS extends AbstractHadoopProcessor {
 final Long maxAgeProp = 
context.getProperty(MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
 maximumAge = (maxAgeProp == null) ? Long.MAX_VALUE : maxAgeProp;
 recurseSubdirs = context.getProperty(RECURSE_SUBDIRS).asBoolean();
-pathFilter = new PathFilter() {
+}
+
+protected long getMinimumAge() {
+return minimumAge;
+}
+
+protected long getMaximumAge() {
+return maximumAge;
+}
+
+public boolean getRecurseSubdirs() {
+return recurseSubdirs;
+}
+
+protected PathFilter getPathFilter(final Path dir) {
+return new PathFilter() {
 
 @Override
 public boolean accept(Path path) {
@@ -512,7 +524,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
 pathToCompare = path.getName();
 } else {
 // 

nifi git commit: NIFI-2828: Fixed issue with transitive Hadoop dependencies in Hive NAR

2016-10-20 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master af27e4070 -> f5f132b1a


NIFI-2828: Fixed issue with transitive Hadoop dependencies in Hive NAR

This closes #1146


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f5f132b1
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f5f132b1
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f5f132b1

Branch: refs/heads/master
Commit: f5f132b1abc63a0b3612e4efe6a3faa129c337e2
Parents: af27e40
Author: Matt Burgess 
Authored: Tue Oct 18 16:22:15 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Oct 20 14:56:01 2016 -0400

--
 .../nifi-hive-bundle/nifi-hive-processors/pom.xml | 7 +++
 nifi-nar-bundles/nifi-hive-bundle/pom.xml | 5 +
 pom.xml   | 1 +
 3 files changed, 13 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/f5f132b1/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
index 3d07b86..eebdcbf 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/pom.xml
@@ -58,6 +58,7 @@
 
 org.apache.hadoop
 hadoop-client
+${hive.hadoop.version}
 
 
 com.google.code.findbugs
@@ -68,6 +69,12 @@
 
 org.apache.nifi
 nifi-hadoop-utils
+
+
+org.apache.hadoop
+hadoop-client
+
+
 
 
 com.github.stephenc.findbugs

http://git-wip-us.apache.org/repos/asf/nifi/blob/f5f132b1/nifi-nar-bundles/nifi-hive-bundle/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-hive-bundle/pom.xml 
b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
index 5ab10e8..0186740 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
@@ -26,6 +26,11 @@
 1.1.0-SNAPSHOT
 pom
 
+
+
+${hive.hadoop.version}
+
+
 
 nifi-hive-processors
 nifi-hive-nar

http://git-wip-us.apache.org/repos/asf/nifi/blob/f5f132b1/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 735a9ec..7cf41b4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,6 +101,7 @@ language governing permissions and limitations under the 
License. -->
 2.2.0
 0.6.0
 1.2.1
+2.6.2
 1.1.2
 1.0.1
 



nifi git commit: NIFI-2894: Fixed typo in PutSQL documentation

2016-10-14 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master d63e675a7 -> 596b98865


NIFI-2894: Fixed typo in PutSQL documentation

This closes #1129


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/596b9886
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/596b9886
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/596b9886

Branch: refs/heads/master
Commit: 596b98865b1c3526072ea120f86f0ff9627f82a8
Parents: d63e675
Author: Matt Burgess 
Authored: Thu Oct 13 09:19:08 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Oct 14 12:45:34 2016 -0400

--
 .../src/main/java/org/apache/nifi/processors/standard/PutSQL.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/596b9886/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
index eb27d40..adfac05 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
@@ -116,7 +116,7 @@ public class PutSQL extends AbstractProcessor {
 static final PropertyDescriptor SUPPORT_TRANSACTIONS = new 
PropertyDescriptor.Builder()
 .name("Support Fragmented Transactions")
 .description("If true, when a FlowFile is consumed by this 
Processor, the Processor will first check the fragment.identifier and 
fragment.count attributes of that FlowFile. "
-+ "If the fragment.count value is greater than 1, the 
Processor will not process any FlowFile will that fragment.identifier until all 
are available; "
++ "If the fragment.count value is greater than 1, the 
Processor will not process any FlowFile with that fragment.identifier until all 
are available; "
 + "at that point, it will process all FlowFiles with that 
fragment.identifier as a single transaction, in the order specified by the 
FlowFiles' fragment.index attributes. "
 + "This Provides atomicity of those SQL statements. If 
this value is false, these attributes will be ignored and the updates will 
occur independent of one another.")
 .allowableValues("true", "false")



nifi git commit: NIFI-2901: Comment out hadoop.version overrides for vendor profiles

2016-10-14 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master b52b83989 -> d63e675a7


NIFI-2901: Comment out hadoop.version overrides for vendor profiles

This closes #1139


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d63e675a
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d63e675a
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d63e675a

Branch: refs/heads/master
Commit: d63e675a7d2905ae8408adc8114c01c2ea4604c8
Parents: b52b839
Author: Matt Burgess 
Authored: Fri Oct 14 12:05:52 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Oct 14 12:40:31 2016 -0400

--
 pom.xml | 9 ++---
 1 file changed, 6 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/d63e675a/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 497823b..735a9ec 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1869,8 +1869,9 @@ language governing permissions and limitations under the 
License. -->
 
 
 
+ command-line
 2.7.1.2.4.0.0-169
+-->
 
 
 
@@ -1892,8 +1893,9 @@ language governing permissions and limitations under the 
License. -->
 
 
 
+ command-line
 2.7.0-mapr-1602
+-->
 
 
 
@@ -1915,8 +1917,9 @@ language governing permissions and limitations under the 
License. -->
 
 
 
+ command-line
 2.6.0-cdh5.8.1
+-->
 
 
 



nifi git commit: NIFI-2897: Fixed SelectHiveQL for CSV output of complex types

2016-10-14 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 88d125137 -> b52b83989


NIFI-2897: Fixed SelectHiveQL for CSV output of complex types

This closes #1132


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/b52b8398
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/b52b8398
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/b52b8398

Branch: refs/heads/master
Commit: b52b8398957dcb60b93ea5f697f45d3e0e212082
Parents: 88d1251
Author: Matt Burgess 
Authored: Thu Oct 13 13:27:49 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Oct 14 12:35:38 2016 -0400

--
 .../java/org/apache/nifi/util/hive/HiveJdbcCommon.java| 10 ++
 1 file changed, 10 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/b52b8398/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
--
diff --git 
a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
index d4b2945..689baf9 100644
--- 
a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
+++ 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java
@@ -339,6 +339,16 @@ public class HiveJdbcCommon {
 rowValues.add("");
 }
 break;
+case ARRAY:
+case STRUCT:
+case JAVA_OBJECT:
+String complexValueString = rs.getString(i);
+if (complexValueString != null) {
+
rowValues.add(StringEscapeUtils.escapeCsv(complexValueString));
+} else {
+rowValues.add("");
+}
+break;
 default:
 if (value != null) {
 rowValues.add(value.toString());



nifi git commit: Removing reference to removed lineage identifier header for ExecuteFlumeSink additional details.

2016-09-21 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master e10b4beb9 -> fda15d916


Removing reference to removed lineage identifier header for ExecuteFlumeSink 
additional details.

THis closes #1039


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/fda15d91
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/fda15d91
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/fda15d91

Branch: refs/heads/master
Commit: fda15d916df12e627651469d2002eb84a72d77f4
Parents: e10b4be
Author: Aldrin Piri 
Authored: Tue Sep 20 13:49:18 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Sep 21 10:36:24 2016 -0400

--
 .../additionalDetails.html   | 4 
 1 file changed, 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/fda15d91/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
--
diff --git 
a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
 
b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
index be8b99f..56e9e99 100644
--- 
a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
+++ 
b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
@@ -46,10 +46,6 @@
 FlowFile#getLastQueueDate()
 
 
-nifi.lineage.identifiers.${i}
-FlowFile#getLineageIdentifiers()[i]
-
-
 nifi.lineage.start.date
 FlowFile#getLineageStartDate()
 



nifi git commit: NIFI-1170 - Improved TailFile processor to support multiple files tailing

2016-09-21 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 68291636c -> 930e95aa0


NIFI-1170 - Improved TailFile processor to support multiple files tailing

This closes #980


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/930e95aa
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/930e95aa
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/930e95aa

Branch: refs/heads/master
Commit: 930e95aa0023b12e5618068ea144808e5627cea7
Parents: 6829163
Author: Pierre Villard 
Authored: Wed Aug 31 18:53:26 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Wed Sep 21 10:05:39 2016 -0400

--
 .../nifi/processors/standard/TailFile.java  | 577 +++
 .../additionalDetails.html  | 134 +
 .../nifi/processors/standard/TestTailFile.java  | 316 +-
 3 files changed, 916 insertions(+), 111 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/930e95aa/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
index 16bde4c..ba2c9b8 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
@@ -29,29 +29,38 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
 import java.util.zip.CRC32;
 import java.util.zip.CheckedInputStream;
 import java.util.zip.Checksum;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.Stateful;
 import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.flowfile.FlowFile;
@@ -71,7 +80,7 @@ import org.apache.nifi.stream.io.StreamUtils;
 @TriggerSerially
 @InputRequirement(Requirement.INPUT_FORBIDDEN)
 @Tags({"tail", "file", "log", "text", "source"})
-@CapabilityDescription("\"Tails\" a file, ingesting data from the file as it 
is written to the file. The file is expected to be textual. Data is ingested 
only when a "
+@CapabilityDescription("\"Tails\" a file, or a list of files, ingesting data 
from the file as it is written to the file. The file is expected to be textual. 
Data is ingested only when a "
 + "new line is encountered (carriage return or new-line character or 
combination). If the file to tail is periodically \"rolled over\", as is 
generally the case "
 + "with log files, an optional Rolling Filename Pattern can be used to 
retrieve data from files that have rolled over, even if the rollover occurred 
while NiFi "
 + "was not running (provided that the data still exists upon restart 
of NiFi). It is generally advisable to set the Run Schedule to a few seconds, 
rather than running "
@@ -79,35 +88,83 @@ import org.apache.nifi.stream.io.StreamUtils;
 + "ingesting files that have been compressed when 'rolled over'.")
 @Stateful(scopes = {Scope.LOCAL, Scope.CLUSTER}, description = "Stores state 
about where in the Tailed File it left off so that on restart it does not have 
to 

[2/2] nifi git commit: NIFI-2789, NIFI-2790 - polishing

2016-09-20 Thread ozhurakousky
NIFI-2789, NIFI-2790 - polishing


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c2e98f96
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c2e98f96
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c2e98f96

Branch: refs/heads/0.x
Commit: c2e98f96ebe30a6a8c919ed8a747a5829f5dea0f
Parents: 58fdfdc
Author: Oleg Zhurakousky 
Authored: Tue Sep 20 10:18:33 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Tue Sep 20 10:20:31 2016 -0400

--
 .../jms/processors/AbstractJMSProcessor.java |  4 ++--
 .../apache/nifi/jms/processors/ConsumeJMS.java   | 16 ++--
 .../apache/nifi/jms/processors/JMSConsumer.java  |  2 +-
 .../apache/nifi/jms/processors/JMSPublisher.java |  7 +++
 .../apache/nifi/jms/processors/PublishJMS.java   |  9 -
 .../nifi/jms/processors/ConsumeJMSTest.java  | 19 +--
 6 files changed, 25 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c2e98f96/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index ed45b84..54e2d89 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -66,9 +66,10 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 .build();
 static final PropertyDescriptor DESTINATION = new 
PropertyDescriptor.Builder()
 .name("Destination Name")
-.description("The name of the JMS Destination. Usually provided by 
the administrator (e.g., 'topic://myTopic').")
+.description("The name of the JMS Destination. Usually provided by 
the administrator (e.g., 'topic://myTopic' or 'myTopic').")
 .required(true)
 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(true)
 .build();
 static final PropertyDescriptor DESTINATION_TYPE = new 
PropertyDescriptor.Builder()
 .name("Destination Type")
@@ -198,7 +199,6 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 
 JmsTemplate jmsTemplate = new JmsTemplate();
 jmsTemplate.setConnectionFactory(this.cachingConnectionFactory);
-this.destinationName = 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
 
jmsTemplate.setPubSubDomain(TOPIC.equals(context.getProperty(DESTINATION_TYPE).getValue()));
 
 // set of properties that may be good candidates for exposure via 
configuration

http://git-wip-us.apache.org/repos/asf/nifi/blob/c2e98f96/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index e8e0eb9..131d113 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -50,7 +50,7 @@ import org.springframework.jms.core.JmsTemplate;
 @Tags({ "jms", "get", "message", "receive", "consume" })
 @InputRequirement(Requirement.INPUT_FORBIDDEN)
 @CapabilityDescription("Consumes JMS Message of type BytesMessage or 
TextMessage transforming its content to "
-+ "a FlowFile and transitioning it to 'success' relationship.")
++ "a FlowFile and transitioning it to 'success' relationship. JMS 
attributes such as headers and properties will be copied as FlowFile 
attributes.")
 @SeeAlso(value = { PublishJMS.class, JMSConnectionFactoryProvider.class })
 public class ConsumeJMS extends AbstractJMSProcessor {
 
@@ -88,8 +88,8 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
 });
 Map jmsHeaders = response.getMessageHeaders();
 Map jmsProperties = Collections.unmodifiableMap(response.getMessageProperties());
-   

[1/2] nifi git commit: NIFI-2789, NIFI-2790 - Read JMS properties and add to FlowFile attributes in ConsumeJMS

2016-09-20 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 639e6d6a7 -> c2e98f96e


NIFI-2789, NIFI-2790 - Read JMS properties and add to FlowFile attributes in 
ConsumeJMS

Remove unused assertEquals import

Move destination from default to send/receive to support EL better


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/58fdfdc7
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/58fdfdc7
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/58fdfdc7

Branch: refs/heads/0.x
Commit: 58fdfdc76e2e59bffbbcdd4e96ad2a190b412b29
Parents: 639e6d6
Author: Joey Frazee 
Authored: Mon Sep 19 08:18:30 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Tue Sep 20 10:11:27 2016 -0400

--
 .../jms/processors/AbstractJMSProcessor.java|  2 +-
 .../apache/nifi/jms/processors/ConsumeJMS.java  | 15 ---
 .../apache/nifi/jms/processors/JMSConsumer.java |  4 +-
 .../nifi/jms/processors/JMSPublisher.java   | 10 ++---
 .../apache/nifi/jms/processors/PublishJMS.java  |  3 +-
 .../apache/nifi/jms/processors/CommonTest.java  |  3 +-
 .../nifi/jms/processors/ConsumeJMSTest.java | 32 ---
 .../processors/JMSPublisherConsumerTest.java| 31 ---
 .../nifi/jms/processors/PublishJMSTest.java | 42 ++--
 9 files changed, 101 insertions(+), 41 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/58fdfdc7/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index f8030db..ed45b84 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -198,7 +198,7 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 
 JmsTemplate jmsTemplate = new JmsTemplate();
 jmsTemplate.setConnectionFactory(this.cachingConnectionFactory);
-
jmsTemplate.setDefaultDestinationName(context.getProperty(DESTINATION).getValue());
+this.destinationName = 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
 
jmsTemplate.setPubSubDomain(TOPIC.equals(context.getProperty(DESTINATION_TYPE).getValue()));
 
 // set of properties that may be good candidates for exposure via 
configuration

http://git-wip-us.apache.org/repos/asf/nifi/blob/58fdfdc7/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index a4cad0d..e8e0eb9 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -76,7 +76,8 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
  */
 @Override
 protected void rendezvousWithJms(ProcessContext context, ProcessSession 
processSession) throws ProcessException {
-final JMSResponse response = this.targetResource.consume();
+final String destinationName = 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
+final JMSResponse response = 
this.targetResource.consume(destinationName);
 if (response != null){
 FlowFile flowFile = processSession.create();
 flowFile = processSession.write(flowFile, new 
OutputStreamCallback() {
@@ -86,8 +87,10 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
 }
 });
 Map jmsHeaders = response.getMessageHeaders();
-flowFile = this.updateFlowFileAttributesWithJmsHeaders(jmsHeaders, 
flowFile, processSession);
-processSession.getProvenanceReporter().receive(flowFile, 
context.getProperty(DESTINATION).getValue());
+Map jmsProperties = Collections.unmodifiableMap(response.getMessageProperties());
+flowFile = 

[1/2] nifi git commit: NIFI-2789, NIFI-2790 - Read JMS properties and add to FlowFile attributes in ConsumeJMS

2016-09-20 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master feaa4c9db -> b693a4a56


NIFI-2789, NIFI-2790 - Read JMS properties and add to FlowFile attributes in 
ConsumeJMS

Remove unused assertEquals import

Move destination from default to send/receive to support EL better


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c2386760
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c2386760
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c2386760

Branch: refs/heads/master
Commit: c23867605857f8380140ba4d69437b58585e9cba
Parents: feaa4c9
Author: Joey Frazee 
Authored: Mon Sep 19 08:18:30 2016 -0500
Committer: Oleg Zhurakousky 
Committed: Tue Sep 20 09:30:14 2016 -0400

--
 .../jms/processors/AbstractJMSProcessor.java|  1 -
 .../apache/nifi/jms/processors/ConsumeJMS.java  | 13 +++---
 .../apache/nifi/jms/processors/JMSConsumer.java |  4 +-
 .../nifi/jms/processors/JMSPublisher.java   | 10 ++---
 .../apache/nifi/jms/processors/PublishJMS.java  |  3 +-
 .../apache/nifi/jms/processors/CommonTest.java  |  3 +-
 .../nifi/jms/processors/ConsumeJMSTest.java | 21 +++---
 .../processors/JMSPublisherConsumerTest.java| 30 +++---
 .../nifi/jms/processors/PublishJMSTest.java | 42 ++--
 9 files changed, 89 insertions(+), 38 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c2386760/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index 20937b5..d7c40f7 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -202,7 +202,6 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 JmsTemplate jmsTemplate = new JmsTemplate();
 jmsTemplate.setConnectionFactory(this.cachingConnectionFactory);
 this.destinationName = 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
-jmsTemplate.setDefaultDestinationName(this.destinationName);
 
jmsTemplate.setPubSubDomain(TOPIC.equals(context.getProperty(DESTINATION_TYPE).getValue()));
 
 // set of properties that may be good candidates for exposure via 
configuration

http://git-wip-us.apache.org/repos/asf/nifi/blob/c2386760/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index 83e594a..cdd5fcd 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -78,7 +78,8 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
  */
 @Override
 protected void rendezvousWithJms(ProcessContext context, ProcessSession 
processSession) throws ProcessException {
-final JMSResponse response = this.targetResource.consume();
+final String destinationName = 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
+final JMSResponse response = 
this.targetResource.consume(destinationName);
 if (response != null){
 FlowFile flowFile = processSession.create();
 flowFile = processSession.write(flowFile, new 
OutputStreamCallback() {
@@ -88,7 +89,9 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
 }
 });
 Map jmsHeaders = response.getMessageHeaders();
-flowFile = this.updateFlowFileAttributesWithJmsHeaders(jmsHeaders, 
flowFile, processSession);
+Map jmsProperties = Collections.unmodifiableMap(response.getMessageProperties());
+flowFile = this.updateFlowFileAttributesWithMap(jmsHeaders, 
flowFile, processSession);
+flowFile = this.updateFlowFileAttributesWithMap(jmsProperties, 

[2/2] nifi git commit: NIFI-2789, NIFI-2790 - polishing

2016-09-20 Thread ozhurakousky
NIFI-2789, NIFI-2790 - polishing

This closes #1027


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/b693a4a5
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/b693a4a5
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/b693a4a5

Branch: refs/heads/master
Commit: b693a4a5611b7ef48787cfa9cdde0bbffc95939f
Parents: c238676
Author: Oleg Zhurakousky 
Authored: Tue Sep 20 09:29:00 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Tue Sep 20 09:32:43 2016 -0400

--
 .../nifi/jms/processors/AbstractJMSProcessor.java   |  2 +-
 .../org/apache/nifi/jms/processors/ConsumeJMS.java  | 16 ++--
 .../org/apache/nifi/jms/processors/JMSConsumer.java |  2 +-
 .../apache/nifi/jms/processors/JMSPublisher.java|  9 -
 .../org/apache/nifi/jms/processors/PublishJMS.java  |  7 +++
 5 files changed, 19 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/b693a4a5/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index d7c40f7..cae3dc2 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -66,7 +66,7 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 .build();
 static final PropertyDescriptor DESTINATION = new 
PropertyDescriptor.Builder()
 .name("Destination Name")
-.description("The name of the JMS Destination. Usually provided by 
the administrator (e.g., 'topic://myTopic').")
+.description("The name of the JMS Destination. Usually provided by 
the administrator (e.g., 'topic://myTopic' or 'myTopic').")
 .required(true)
 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
 .expressionLanguageSupported(true)

http://git-wip-us.apache.org/repos/asf/nifi/blob/b693a4a5/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index cdd5fcd..aea6c9c 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -50,7 +50,7 @@ import org.springframework.jms.core.JmsTemplate;
 @Tags({ "jms", "get", "message", "receive", "consume" })
 @InputRequirement(Requirement.INPUT_FORBIDDEN)
 @CapabilityDescription("Consumes JMS Message of type BytesMessage or 
TextMessage transforming its content to "
-+ "a FlowFile and transitioning it to 'success' relationship.")
++ "a FlowFile and transitioning it to 'success' relationship. JMS 
attributes such as headers and properties will be copied as FlowFile 
attributes.")
 @SeeAlso(value = { PublishJMS.class, JMSConnectionFactoryProvider.class })
 public class ConsumeJMS extends AbstractJMSProcessor {
 
@@ -90,8 +90,8 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
 });
 Map jmsHeaders = response.getMessageHeaders();
 Map jmsProperties = Collections.unmodifiableMap(response.getMessageProperties());
-flowFile = this.updateFlowFileAttributesWithMap(jmsHeaders, 
flowFile, processSession);
-flowFile = this.updateFlowFileAttributesWithMap(jmsProperties, 
flowFile, processSession);
+flowFile = 
this.updateFlowFileAttributesWithJMSAttributes(jmsHeaders, flowFile, 
processSession);
+flowFile = 
this.updateFlowFileAttributesWithJMSAttributes(jmsProperties, flowFile, 
processSession);
 processSession.getProvenanceReporter().receive(flowFile, 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue());
 processSession.transfer(flowFile, REL_SUCCESS);
 } else {
@@ -116,11 +116,15 @@ public class ConsumeJMS extends 
AbstractJMSProcessor 

nifi git commit: NIFI-2771: - Using GzipHandler instead of GzipFilter.

2016-09-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master aa933a194 -> abcfbeb06


NIFI-2771:
- Using GzipHandler instead of GzipFilter.

This closes #1020


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/abcfbeb0
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/abcfbeb0
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/abcfbeb0

Branch: refs/heads/master
Commit: abcfbeb062ff014e3ffefaaa1a689e0646af1d42
Parents: aa933a1
Author: Matt Gilman 
Authored: Wed Sep 14 10:13:16 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Sep 15 10:27:39 2016 -0400

--
 .../org/apache/nifi/web/server/JettyServer.java | 67 
 .../src/main/webapp/WEB-INF/web.xml | 12 
 2 files changed, 40 insertions(+), 39 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/abcfbeb0/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
index 10b6513..8180c75 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
@@ -16,32 +16,6 @@
  */
 package org.apache.nifi.web.server;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-
-import javax.servlet.DispatcherType;
-import javax.servlet.ServletContext;
-
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -72,6 +46,7 @@ import org.eclipse.jetty.server.handler.ContextHandler;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerCollection;
 import org.eclipse.jetty.server.handler.ResourceHandler;
+import org.eclipse.jetty.server.handler.gzip.GzipHandler;
 import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.util.resource.Resource;
 import org.eclipse.jetty.util.resource.ResourceCollection;
@@ -88,6 +63,31 @@ import org.springframework.context.ApplicationContext;
 import org.springframework.web.context.WebApplicationContext;
 import org.springframework.web.context.support.WebApplicationContextUtils;
 
+import javax.servlet.DispatcherType;
+import javax.servlet.ServletContext;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
 /**
  * Encapsulates the Jetty instance.
  */
@@ -326,7 +326,20 @@ public class JettyServer implements NiFiServer {
 handlers.addHandler(loadWar(webErrorWar, "/", frameworkClassLoader));
 
 // deploy the web apps
-server.setHandler(handlers);
+server.setHandler(gzip(handlers));
+}
+
+/**
+ * Enables compression for the specified handler.
+ *
+ * @param handler handler to enable compression for
+ * @return compression enabled handler
+ */
+private Handler gzip(final Handler handler) {
+final GzipHandler gzip = new GzipHandler();
+gzip.setIncludedMethods("GET", "POST", "PUT", "DELETE");
+gzip.setHandler(handler);
+return gzip;
 }
 
 private Map findWars(final Set narWorkingDirectories) {


nifi git commit: NIFI-2755 - Fixes minor typo in Developers Guide

2016-09-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master c1ad91359 -> aa933a194


NIFI-2755 - Fixes minor typo in Developers Guide

This closes #1003


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/aa933a19
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/aa933a19
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/aa933a19

Branch: refs/heads/master
Commit: aa933a194110b58a8c2fe07013f37a6fc1d10e71
Parents: c1ad913
Author: Andre F de Miranda 
Authored: Mon Sep 12 01:04:11 2016 +1000
Committer: Oleg Zhurakousky 
Committed: Thu Sep 15 10:12:04 2016 -0400

--
 nifi-docs/src/main/asciidoc/developer-guide.adoc | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/aa933a19/nifi-docs/src/main/asciidoc/developer-guide.adoc
--
diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc 
b/nifi-docs/src/main/asciidoc/developer-guide.adoc
index c9bb1fb..de5fb46 100644
--- a/nifi-docs/src/main/asciidoc/developer-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc
@@ -1744,7 +1744,8 @@ have an `onTrigger` method because Controller Services 
are not
 scheduled to run periodically, and
 Controller Services do not have Relationships because they are not
 integrated into the flow directly. Rather,
-they are used Processors, Reporting Tasks, and other Controller Services.
+they are used by Processors, Reporting Tasks, and
+other Controller Services.
 
 [[developing-controller-service]]
 === Developing a ControllerService



nifi git commit: NIFI-1342 Added default User-Agent in PostHttp

2016-09-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 3e70fbde3 -> e5761966c


NIFI-1342 Added default User-Agent in PostHttp

This closes #1021


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/e5761966
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/e5761966
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/e5761966

Branch: refs/heads/0.x
Commit: e5761966c9576143ce3e07776687e34e981025d0
Parents: 3e70fbd
Author: Pierre Villard 
Authored: Wed Sep 14 17:59:17 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Thu Sep 15 10:05:10 2016 -0400

--
 .../java/org/apache/nifi/processors/standard/PostHTTP.java| 2 ++
 .../org/apache/nifi/processors/standard/TestPostHTTP.java | 7 +++
 2 files changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/e5761966/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
index e78f1e5..2b99369 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
@@ -82,6 +82,7 @@ import 
org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.protocol.HttpCoreContext;
 import org.apache.http.util.EntityUtils;
+import org.apache.http.util.VersionInfo;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
@@ -195,6 +196,7 @@ public class PostHTTP extends AbstractProcessor {
 .description("What to report as the User Agent when we connect to 
the remote server")
 .required(false)
 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.defaultValue(VersionInfo.getUserAgent("Apache-HttpClient", 
"org.apache.http.client", HttpClientBuilder.class))
 .build();
 public static final PropertyDescriptor COMPRESSION_LEVEL = new 
PropertyDescriptor.Builder()
 .name("Compression Level")

http://git-wip-us.apache.org/repos/asf/nifi/blob/e5761966/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
index edff3b4..cfda2ef 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
@@ -434,4 +434,11 @@ public class TestPostHTTP {
 Assert.assertEquals(suppliedMimeType, 
lastPostHeaders.get(PostHTTP.CONTENT_TYPE_HEADER));
 Assert.assertEquals("4700",lastPostHeaders.get("Content-Length"));
 }
+
+@Test
+public void testDefaultUserAgent() throws Exception {
+setup(null);
+
Assert.assertTrue(runner.getProcessContext().getProperty(PostHTTP.USER_AGENT).getValue().startsWith("Apache-HttpClient"));
+}
+
 }



nifi git commit: NIFI-1342 Added default User-Agent in PostHttp

2016-09-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 8a28395e9 -> c1ad91359


NIFI-1342 Added default User-Agent in PostHttp

This closes #1021


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c1ad9135
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c1ad9135
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c1ad9135

Branch: refs/heads/master
Commit: c1ad913598c9c41fdc99a53d9939fece9c3250b9
Parents: 8a28395
Author: Pierre Villard 
Authored: Wed Sep 14 17:59:17 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Thu Sep 15 09:57:09 2016 -0400

--
 .../java/org/apache/nifi/processors/standard/PostHTTP.java| 2 ++
 .../org/apache/nifi/processors/standard/TestPostHTTP.java | 7 +++
 2 files changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c1ad9135/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
index 37b307e..44b7973 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
@@ -82,6 +82,7 @@ import 
org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.protocol.HttpCoreContext;
 import org.apache.http.util.EntityUtils;
+import org.apache.http.util.VersionInfo;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
@@ -194,6 +195,7 @@ public class PostHTTP extends AbstractProcessor {
 .description("What to report as the User Agent when we connect to 
the remote server")
 .required(false)
 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.defaultValue(VersionInfo.getUserAgent("Apache-HttpClient", 
"org.apache.http.client", HttpClientBuilder.class))
 .build();
 public static final PropertyDescriptor COMPRESSION_LEVEL = new 
PropertyDescriptor.Builder()
 .name("Compression Level")

http://git-wip-us.apache.org/repos/asf/nifi/blob/c1ad9135/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
index edff3b4..cfda2ef 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
@@ -434,4 +434,11 @@ public class TestPostHTTP {
 Assert.assertEquals(suppliedMimeType, 
lastPostHeaders.get(PostHTTP.CONTENT_TYPE_HEADER));
 Assert.assertEquals("4700",lastPostHeaders.get("Content-Length"));
 }
+
+@Test
+public void testDefaultUserAgent() throws Exception {
+setup(null);
+
Assert.assertTrue(runner.getProcessContext().getProperty(PostHTTP.USER_AGENT).getValue().startsWith("Apache-HttpClient"));
+}
+
 }



nifi git commit: NIFI-2745 added _source destination name_ attribute to ConsumeJMS

2016-09-07 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 938e32ed9 -> d36b76cc6


NIFI-2745 added _source destination name_ attribute to ConsumeJMS

This closes #992


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d36b76cc
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d36b76cc
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d36b76cc

Branch: refs/heads/master
Commit: d36b76cc600b76748ffba9b94d6b702af552756f
Parents: 938e32e
Author: Oleg Zhurakousky 
Authored: Wed Sep 7 11:04:54 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Sep 7 15:28:34 2016 -0400

--
 .../org/apache/nifi/jms/processors/AbstractJMSProcessor.java| 5 -
 .../main/java/org/apache/nifi/jms/processors/ConsumeJMS.java| 3 +++
 .../java/org/apache/nifi/jms/processors/ConsumeJMSTest.java | 2 ++
 3 files changed, 9 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/d36b76cc/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
index 398c5c1..20937b5 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/AbstractJMSProcessor.java
@@ -114,6 +114,8 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 
 private volatile CachingConnectionFactory cachingConnectionFactory;
 
+protected volatile String destinationName;
+
 /**
  *
  */
@@ -199,7 +201,8 @@ abstract class AbstractJMSProcessor 
extends AbstractProcess
 
 JmsTemplate jmsTemplate = new JmsTemplate();
 jmsTemplate.setConnectionFactory(this.cachingConnectionFactory);
-
jmsTemplate.setDefaultDestinationName(context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue());
+this.destinationName = 
context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
+jmsTemplate.setDefaultDestinationName(this.destinationName);
 
jmsTemplate.setPubSubDomain(TOPIC.equals(context.getProperty(DESTINATION_TYPE).getValue()));
 
 // set of properties that may be good candidates for exposure via 
configuration

http://git-wip-us.apache.org/repos/asf/nifi/blob/d36b76cc/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
index ac05f2c..83e594a 100644
--- 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
+++ 
b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java
@@ -54,6 +54,8 @@ import org.springframework.jms.core.JmsTemplate;
 @SeeAlso(value = { PublishJMS.class, JMSConnectionFactoryProvider.class })
 public class ConsumeJMS extends AbstractJMSProcessor {
 
+public static final String JMS_SOURCE_DESTINATION_NAME = 
"jms.source.destination";
+
 public static final Relationship REL_SUCCESS = new Relationship.Builder()
 .name("success")
 .description("All FlowFiles that are received from the JMS 
Destination are routed to this relationship")
@@ -118,6 +120,7 @@ public class ConsumeJMS extends 
AbstractJMSProcessor {
 for (Entry headersEntry : jmsHeaders.entrySet()) {
 attributes.put(headersEntry.getKey(), 
String.valueOf(headersEntry.getValue()));
 }
+attributes.put(JMS_SOURCE_DESTINATION_NAME, this.destinationName);
 flowFile = processSession.putAllAttributes(flowFile, attributes);
 return flowFile;
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d36b76cc/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/ConsumeJMSTest.java
 

nifi git commit: NIFI-2628: Whenever a flow is inherited from cluster coordinator, save flow immediately to disk

2016-08-30 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master ba1923875 -> 1213b4663


NIFI-2628: Whenever a flow is inherited from cluster coordinator, save flow 
immediately to disk

This closes #967


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/1213b466
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/1213b466
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/1213b466

Branch: refs/heads/master
Commit: 1213b46632e29ba8fd6f46a1633c8be23c352a82
Parents: ba19238
Author: Mark Payne 
Authored: Mon Aug 29 14:38:30 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Tue Aug 30 16:13:45 2016 -0400

--
 .../main/java/org/apache/nifi/controller/StandardFlowService.java   | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/1213b466/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
index af96cfd..417a994 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
@@ -491,6 +491,7 @@ public class StandardFlowService implements FlowService, 
ProtocolHandler {
 } else {
 try {
 loadFromConnectionResponse(response);
+dao.save(controller, true);
 } catch (final Exception e) {
 logger.error("Failed to load flow from cluster due to: 
" + e, e);
 handleConnectionFailure(e);



nifi git commit: NIFI-2643 Changed null check to !isBlank() for key in RunNiFi.

2016-08-24 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 487f05a2b -> 6331dda8d


NIFI-2643 Changed null check to !isBlank() for key in RunNiFi.

NIFI-2643 Updated comment in bootstrap.conf template.

This closes #932


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6331dda8
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6331dda8
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6331dda8

Branch: refs/heads/master
Commit: 6331dda8d3c3fd195551f23479069864845cd6bd
Parents: 487f05a
Author: Andy LoPresto 
Authored: Wed Aug 24 12:17:14 2016 -0700
Committer: Oleg Zhurakousky 
Committed: Wed Aug 24 16:20:10 2016 -0400

--
 .../src/main/java/org/apache/nifi/bootstrap/RunNiFi.java   | 2 +-
 .../nifi-resources/src/main/resources/conf/bootstrap.conf  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/6331dda8/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
--
diff --git 
a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java 
b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index eb3d8ec..04e7ba3 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -1021,7 +1021,7 @@ public class RunNiFi {
 cmd.add("-Dapp=NiFi");
 cmd.add("-Dorg.apache.nifi.bootstrap.config.log.dir=" + nifiLogDir);
 cmd.add("org.apache.nifi.NiFi");
-if (props.containsKey(NIFI_BOOTSTRAP_SENSITIVE_KEY) && 
props.get(NIFI_BOOTSTRAP_SENSITIVE_KEY) != null) {
+if (props.containsKey(NIFI_BOOTSTRAP_SENSITIVE_KEY) && 
!StringUtils.isBlank(props.get(NIFI_BOOTSTRAP_SENSITIVE_KEY))) {
 cmd.add("-k " + props.get(NIFI_BOOTSTRAP_SENSITIVE_KEY));
 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/6331dda8/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
index f4a6c29..67c3122 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
@@ -51,7 +51,7 @@ java.arg.13=-XX:+UseG1GC
 #Set headless mode by default
 java.arg.14=-Djava.awt.headless=true
 
-#Set sensitive properties key
+# Master key in hexadecimal format for encrypted sensitive configuration values
 nifi.bootstrap.sensitive.key=
 
 ###



nifi git commit: NIFI-2642 Catches (and ignores) ResourceNotFoundException during the updating of a property value for a controller service when the current value points to a controller service that h

2016-08-24 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master c2ae7a6d7 -> 6475924f5


NIFI-2642 Catches (and ignores) ResourceNotFoundException during the updating 
of a property value for a controller service when the current value points to a 
controller service that has been deleted, allowing the assigning of a new 
controller service to continue.

This closes #931


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6475924f
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6475924f
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6475924f

Branch: refs/heads/master
Commit: 6475924f53404939214c7519f5e411ffa1ed27c0
Parents: c2ae7a6
Author: Jeff Storck 
Authored: Wed Aug 24 15:06:01 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Aug 24 15:55:04 2016 -0400

--
 .../authorization/AuthorizeControllerServiceReference.java  | 9 +++--
 1 file changed, 7 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/6475924f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
index 8617135..48c782f 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
@@ -20,6 +20,7 @@ import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.web.ResourceNotFoundException;
 
 import java.util.Map;
 import java.util.Objects;
@@ -57,8 +58,12 @@ public final class AuthorizeControllerServiceReference {
 if (!Objects.equals(currentValue, proposedValue)) {
 // ensure access to the old service
 if (currentValue != null) {
-final Authorizable currentServiceAuthorizable = 
lookup.getControllerService(currentValue).getAuthorizable();
-currentServiceAuthorizable.authorize(authorizer, 
RequestAction.READ, user);
+try {
+final Authorizable currentServiceAuthorizable 
= lookup.getControllerService(currentValue).getAuthorizable();
+
currentServiceAuthorizable.authorize(authorizer, RequestAction.READ, user);
+} catch (ResourceNotFoundException e) {
+// ignore if the resource is not found, if 
currentValue was previously deleted, it should not stop assignment of 
proposedValue
+}
 }
 
 // ensure access to the new service



[1/2] nifi git commit: NIFI-2519 - ListenSMTP addresses mismatch between smtp.src attribute documentation and attribute.put

2016-08-17 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 0ffdc2eb9 -> a0fab15eb


NIFI-2519 - ListenSMTP addresses mismatch between smtp.src attribute 
documentation and attribute.put


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/e7288af8
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/e7288af8
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/e7288af8

Branch: refs/heads/master
Commit: e7288af859d00d6a4d12631d73a92404e31ba7f4
Parents: 0ffdc2e
Author: Andre F de Miranda 
Authored: Tue Aug 16 09:13:39 2016 +1000
Committer: Oleg Zhurakousky 
Committed: Wed Aug 17 14:58:02 2016 -0400

--
 .../src/main/java/org/apache/nifi/processors/email/ListenSMTP.java | 2 +-
 .../java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/e7288af8/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
--
diff --git 
a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 
b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
index 760db13..9e422ca 100644
--- 
a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
+++ 
b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
@@ -71,7 +71,7 @@ import org.subethamail.smtp.server.SMTPServer;
 + "certificates used by an TLS peer"),
 @WritesAttribute(attribute = "smtp.certificates.*.principal", description 
= "The principal for each of the "
 + "certificates used by an TLS peer"),
-@WritesAttribute(attribute = "smtp.src", description = "The source IP of 
the SMTP connection"),
+@WritesAttribute(attribute = "smtp.src", description = "The source IP and 
port of the SMTP connection"),
 @WritesAttribute(attribute = "smtp.from", description = "The value used 
during MAIL FROM (i.e. envelope)"),
 @WritesAttribute(attribute = "smtp.recipient.*", description = "The values 
used during RCPT TO (i.e. envelope)"),
 @WritesAttribute(attribute = "mime.type", description = "Mime type of the 
message")})

http://git-wip-us.apache.org/repos/asf/nifi/blob/e7288af8/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
--
diff --git 
a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
 
b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
index 4dad3bc..a6d7b33 100644
--- 
a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
+++ 
b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
@@ -149,7 +149,7 @@ public class SmtpConsumer implements MessageHandler {
 }
 
 attributes.put("smtp.helo", context.getHelo());
-attributes.put("smtp.remote.addr", 
context.getRemoteAddress().toString());
+attributes.put("smtp.src", 
context.getRemoteAddress().toString().substring(1));
 attributes.put("smtp.from", from);
 for (int i = 0; i < recipientList.size(); i++) {
 attributes.put("smtp.recipient." + i, recipientList.get(i));



[2/2] nifi git commit: NIFI-2519 polishing remote address formatting to avoid NPE and other potential exceptions.

2016-08-17 Thread ozhurakousky
NIFI-2519 polishing remote address formatting to avoid NPE and other potential 
exceptions.

This closes #871


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/a0fab15e
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/a0fab15e
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/a0fab15e

Branch: refs/heads/master
Commit: a0fab15eb747426dd4be8cae5378715dd80f6d55
Parents: e7288af
Author: Oleg Zhurakousky 
Authored: Wed Aug 17 14:57:42 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Aug 17 15:13:05 2016 -0400

--
 .../nifi/processors/email/smtp/SmtpConsumer.java  | 14 --
 1 file changed, 12 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/a0fab15e/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
--
diff --git 
a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
 
b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
index a6d7b33..10eea88 100644
--- 
a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
+++ 
b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/SmtpConsumer.java
@@ -19,6 +19,8 @@ package org.apache.nifi.processors.email.smtp;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.security.cert.Certificate;
 import java.security.cert.X509Certificate;
 import java.util.ArrayList;
@@ -28,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -38,7 +41,6 @@ import 
org.apache.nifi.processor.exception.FlowFileAccessException;
 import org.apache.nifi.processors.email.ListenSMTP;
 import org.apache.nifi.stream.io.LimitingInputStream;
 import org.apache.nifi.util.StopWatch;
-
 import org.subethamail.smtp.MessageContext;
 import org.subethamail.smtp.MessageHandler;
 import org.subethamail.smtp.RejectException;
@@ -148,8 +150,16 @@ public class SmtpConsumer implements MessageHandler {
 }
 }
 
+SocketAddress address = context.getRemoteAddress();
+if (address != null) {
+// will extract and format source address if available
+String strAddress = address instanceof InetSocketAddress
+? ((InetSocketAddress) address).getHostString() + ":" + 
((InetSocketAddress) address).getPort()
+: context.getRemoteAddress().toString();
+attributes.put("smtp.src", strAddress);
+}
+
 attributes.put("smtp.helo", context.getHelo());
-attributes.put("smtp.src", 
context.getRemoteAddress().toString().substring(1));
 attributes.put("smtp.from", from);
 for (int i = 0; i < recipientList.size(); i++) {
 attributes.put("smtp.recipient." + i, recipientList.get(i));



nifi git commit: [NIFI-2559] UI - Controller Service update missing icons

2016-08-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master d45114e48 -> 2a9274704


[NIFI-2559] UI - Controller Service update missing icons

This closes #864


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/2a927470
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/2a927470
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/2a927470

Branch: refs/heads/master
Commit: 2a92747046b39d7b3f7604258c1d3bb81a9447bd
Parents: d45114e
Author: Scott Aslan 
Authored: Mon Aug 15 13:16:25 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Mon Aug 15 16:27:12 2016 -0400

--
 .../src/main/webapp/css/common-ui.css   |  14 --
 .../src/main/webapp/css/controller-service.css  |  16 +++
 .../nifi-web-ui/src/main/webapp/css/login.css   |   1 -
 .../nifi-web-ui/src/main/webapp/css/main.css|  23 +++--
 .../main/webapp/css/processor-configuration.css |   8 +++-
 .../css/remote-process-group-configuration.css  |   8 +++-
 .../nifi-web-ui/src/main/webapp/css/shell.css   |  16 ---
 .../nifi-web-ui/src/main/webapp/css/summary.css |  48 ---
 .../src/main/webapp/images/iconAlert.png| Bin 1396 -> 0 bytes
 .../src/main/webapp/images/iconClose.png| Bin 231 -> 0 bytes
 .../main/webapp/images/iconCollapseChildren.png | Bin 571 -> 0 bytes
 .../main/webapp/images/iconCollapseParents.png  | Bin 645 -> 0 bytes
 .../src/main/webapp/images/iconCommit.png   | Bin 445 -> 0 bytes
 .../src/main/webapp/images/iconConnection.png   | Bin 1517 -> 0 bytes
 .../src/main/webapp/images/iconDelete.png   | Bin 670 -> 0 bytes
 .../src/main/webapp/images/iconDetails.png  | Bin 549 -> 0 bytes
 .../main/webapp/images/iconExpandChildren.png   | Bin 521 -> 0 bytes
 .../main/webapp/images/iconExpandParents.png| Bin 585 -> 0 bytes
 .../src/main/webapp/images/iconInfo.png | Bin 550 -> 0 bytes
 .../src/main/webapp/images/iconInputPort.png| Bin 1842 -> 0 bytes
 .../src/main/webapp/images/iconOutputPort.png   | Bin 1658 -> 0 bytes
 .../src/main/webapp/images/iconPopOut.png   | Bin 217 -> 0 bytes
 .../src/main/webapp/images/iconPortRemoved.png  | Bin 456 -> 0 bytes
 .../src/main/webapp/images/iconProcessGroup.png | Bin 1314 -> 0 bytes
 .../src/main/webapp/images/iconProcessor.png| Bin 1446 -> 0 bytes
 .../webapp/images/iconRemoteProcessGroup.png| Bin 674 -> 0 bytes
 .../src/main/webapp/images/iconRun.png  | Bin 538 -> 0 bytes
 .../src/main/webapp/images/iconTwistArrow.png   | Bin 256 -> 0 bytes
 .../propertytable/jquery.propertytable.css  |   3 +-
 .../main/webapp/js/nf/canvas/nf-context-menu.js |   2 +-
 30 files changed, 94 insertions(+), 45 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/2a927470/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
index 9c947d5..9eac226 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/common-ui.css
@@ -263,12 +263,18 @@ div.ajax-loading {
 background-image: url(../images/iconLoading.gif);
 }
 
-div.ajax-complete {
-background-image: url(../images/iconCommit.png);
+div.ajax-complete:before {
+font-family: FontAwesome;
+content: "\f00c";
+font-size: 16px;
+color: #70B59A;
 }
 
-div.ajax-error {
-background-image: url(../images/iconDelete.png);
+div.ajax-error:before {
+font-family: FontAwesome;
+content: "\f1f8";
+font-size: 16px;
+color: #004849;
 }
 
 .refresh-button {

http://git-wip-us.apache.org/repos/asf/nifi/blob/2a927470/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
index e6d2cde..d8c3e0d 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
@@ -39,10 +39,6 @@
 
 /* controller-service settings 

nifi git commit: NIFI-2524 - Fixes to improve handling of missing journal files during rollover/merge execution.Includes:

2016-08-15 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 44057f097 -> fbd320115


NIFI-2524 - Fixes to improve handling of missing journal files during 
rollover/merge execution.Includes:

This closes #840

 Removed partial file check (based on missing first file)
 Added condition to merge if at least one journal files available on disk. If 
all files are  missing from disk that is considered an error.
 Added retry logic to prevent endless thread execution when encountering errors 
(such as missing files).


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/fbd32011
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/fbd32011
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/fbd32011

Branch: refs/heads/master
Commit: fbd3201157883e2606746af448f5f3057d214d1e
Parents: 44057f0
Author: Yolanda M. Davis 
Authored: Thu Aug 11 05:42:19 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Mon Aug 15 14:43:38 2016 -0400

--
 .../PersistentProvenanceRepository.java | 66 ++--
 .../TestPersistentProvenanceRepository.java | 57 +
 2 files changed, 89 insertions(+), 34 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/fbd32011/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
--
diff --git 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
index f8bb667..38722c5 100644
--- 
a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
+++ 
b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
@@ -125,6 +125,7 @@ public class PersistentProvenanceRepository implements 
ProvenanceRepository {
 public static final Pattern LOG_FILENAME_PATTERN = 
Pattern.compile("(\\d+).*\\.prov");
 public static final int MAX_UNDELETED_QUERY_RESULTS = 10;
 public static final int MAX_INDEXING_FAILURE_COUNT = 5; // how many 
indexing failures we will tolerate before skipping indexing for a prov file
+public static final int MAX_JOURNAL_ROLLOVER_RETRIES = 5;
 
 private static final Logger logger = 
LoggerFactory.getLogger(PersistentProvenanceRepository.class);
 
@@ -1267,25 +1268,23 @@ public class PersistentProvenanceRepository implements 
ProvenanceRepository {
 if (!journalsToMerge.isEmpty()) {
 // Run the rollover logic in a background thread.
 final AtomicReference futureReference = new 
AtomicReference<>();
+final AtomicInteger retryAttempts = new 
AtomicInteger(MAX_JOURNAL_ROLLOVER_RETRIES);
 final int recordsWritten = 
recordsWrittenSinceRollover.getAndSet(0);
 final Runnable rolloverRunnable = new Runnable() {
 @Override
 public void run() {
+
+File fileRolledOver = null;
+
 try {
-final File fileRolledOver;
+fileRolledOver = mergeJournals(journalsToMerge, 
getMergeFile(journalsToMerge, storageDir), eventReporter);
+} catch (final IOException ioe) {
+logger.error("Failed to merge Journal Files {} 
into a Provenance Log File due to {}", journalsToMerge, ioe.toString());
+logger.error("", ioe);
+}
 
-try {
-fileRolledOver = 
mergeJournals(journalsToMerge, getMergeFile(journalsToMerge, storageDir), 
eventReporter);
-} catch (final IOException ioe) {
-logger.error("Failed to merge Journal Files {} 
into a Provenance Log File due to {}", journalsToMerge, ioe.toString());
-logger.error("", ioe);
-return;
-}
+if (fileRolledOver != null)  {
 
-if (fileRolledOver == null) {
-logger.debug("Couldn't merge journals. Will 
try again in 10 seconds. journalsToMerge: {}, storageDir: {}", journalsToMerge, 

nifi git commit: NIFI-2540: Exclude from templates the parts of property descriptors that are not necessary. Also ensure that Property Descriptors are not completely removed from Controller Services.

2016-08-10 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master d9720239f -> 85877a73d


NIFI-2540: Exclude from templates the parts of property descriptors that are 
not necessary. Also ensure that Property Descriptors are not completely removed 
from Controller Services.

This closes #828


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/85877a73
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/85877a73
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/85877a73

Branch: refs/heads/master
Commit: 85877a73dc454a4b7f2b567517d345332ccb3b52
Parents: d972023
Author: Mark Payne 
Authored: Wed Aug 10 11:36:21 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Aug 10 13:27:20 2016 -0400

--
 .../nifi/web/api/dto/PropertyDescriptorDTO.java | 24 
 .../apache/nifi/controller/TemplateUtils.java   | 29 ++--
 2 files changed, 38 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/85877a73/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
index dae16f2..500420b 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java
@@ -31,10 +31,10 @@ public class PropertyDescriptorDTO {
 private String description;
 private String defaultValue;
 private List allowableValues;
-private boolean required;
-private boolean sensitive;
-private boolean dynamic;
-private boolean supportsEl;
+private Boolean required;
+private Boolean sensitive;
+private Boolean dynamic;
+private Boolean supportsEl;
 private String identifiesControllerService;
 
 /**
@@ -113,11 +113,11 @@ public class PropertyDescriptorDTO {
 @ApiModelProperty(
 value = "Whether the property is required."
 )
-public boolean isRequired() {
+public Boolean isRequired() {
 return required;
 }
 
-public void setRequired(boolean required) {
+public void setRequired(Boolean required) {
 this.required = required;
 }
 
@@ -127,11 +127,11 @@ public class PropertyDescriptorDTO {
 @ApiModelProperty(
 value = "Whether the property is sensitive and protected whenever 
stored or represented."
 )
-public boolean isSensitive() {
+public Boolean isSensitive() {
 return sensitive;
 }
 
-public void setSensitive(boolean sensitive) {
+public void setSensitive(Boolean sensitive) {
 this.sensitive = sensitive;
 }
 
@@ -141,11 +141,11 @@ public class PropertyDescriptorDTO {
 @ApiModelProperty(
 value = "Whether the property is dynamic (user-defined)."
 )
-public boolean isDynamic() {
+public Boolean isDynamic() {
 return dynamic;
 }
 
-public void setDynamic(boolean dynamic) {
+public void setDynamic(Boolean dynamic) {
 this.dynamic = dynamic;
 }
 
@@ -155,11 +155,11 @@ public class PropertyDescriptorDTO {
 @ApiModelProperty(
 value = "Whether the property supports expression language."
 )
-public boolean getSupportsEl() {
+public Boolean getSupportsEl() {
 return supportsEl;
 }
 
-public void setSupportsEl(boolean supportsEl) {
+public void setSupportsEl(Boolean supportsEl) {
 this.supportsEl = supportsEl;
 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/85877a73/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
index 668872b..aa84594 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateUtils.java
+++ 

[2/3] nifi git commit: NIFI-1663: Add ConvertAvroToORC processor

2016-08-10 Thread ozhurakousky
http://git-wip-us.apache.org/repos/asf/nifi/blob/d9720239/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java
--
diff --git 
a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java
 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java
new file mode 100644
index 000..a98092a
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcFlowFileWriter.java
@@ -0,0 +1,2650 @@
+/*
+ * 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.hadoop.hive.ql.io.orc;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
+import org.apache.hadoop.hive.ql.io.orc.CompressionCodec.Modifier;
+import org.apache.hadoop.hive.ql.io.orc.OrcFile.CompressionStrategy;
+import org.apache.hadoop.hive.ql.io.orc.OrcFile.EncodingStrategy;
+import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndexEntry;
+import org.apache.hadoop.hive.ql.io.orc.OrcProto.StripeStatistics;
+import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type;
+import org.apache.hadoop.hive.ql.io.orc.OrcProto.UserMetadataItem;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import 

[1/3] nifi git commit: NIFI-1663: Add ConvertAvroToORC processor

2016-08-10 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master bb2431270 -> d9720239f


http://git-wip-us.apache.org/repos/asf/nifi/blob/d9720239/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
--
diff --git 
a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
new file mode 100644
index 000..6100bd1
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java
@@ -0,0 +1,289 @@
+/*
+ * 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.hive;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.orc.CompressionKind;
+import org.apache.hadoop.hive.ql.io.orc.OrcFlowFileWriter;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.util.hive.HiveJdbcCommon;
+import org.apache.nifi.util.hive.HiveUtils;
+import org.apache.hadoop.hive.ql.io.orc.NiFiOrcUtils;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * The ConvertAvroToORC processor takes an Avro-formatted flow file as input 
and converts it into ORC format.
+ */
+@SideEffectFree
+@SupportsBatching
+@Tags({"avro", "orc", "hive", "convert"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription("Converts an Avro record into ORC file format. This 
processor provides a direct mapping of an Avro record to an ORC record, such "
++ "that the resulting ORC file will have the same hierarchical 
structure as the Avro document. If an incoming FlowFile contains a stream of "
++ "multiple Avro records, the resultant FlowFile will contain a ORC 
file containing all of the Avro records.  If an incoming FlowFile does "
++ "not contain any records, an empty ORC file is the output.")
+@WritesAttributes({
+@WritesAttribute(attribute = "mime.type", description = "Sets the mime 
type to application/octet-stream"),
+@WritesAttribute(attribute = "filename", description = "Sets the 
filename to the existing filename with the extension replaced by / added to by 
.orc"),
+@WritesAttribute(attribute = "record.count", description = "Sets the 
number of records in the ORC file."),
+@WritesAttribute(attribute = "hive.ddl", description = "Creates a 
partial Hive DDL statement for creating a 

nifi git commit: NIFI-2518: Added unit test showing issue

2016-08-10 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 219234d00 -> 557d6365b


NIFI-2518: Added unit test showing issue

NIFI-2518: Added support for fractional seconds to 
AbstractDatabaseFetchProcessor

This closes #821


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/557d6365
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/557d6365
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/557d6365

Branch: refs/heads/master
Commit: 557d6365bf27ee2271cd9d3a147361acb6cc9c6c
Parents: 219234d
Author: Matt Burgess 
Authored: Tue Aug 9 10:18:26 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Aug 10 11:10:17 2016 -0400

--
 .../AbstractDatabaseFetchProcessor.java | 21 +---
 .../standard/QueryDatabaseTableTest.java| 52 +++-
 2 files changed, 65 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/557d6365/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
index 6182d93..7b30479 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractDatabaseFetchProcessor.java
@@ -35,6 +35,7 @@ import java.sql.Statement;
 import java.sql.Timestamp;
 import java.text.DecimalFormat;
 import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
@@ -135,6 +136,8 @@ public abstract class AbstractDatabaseFetchProcessor 
extends AbstractSessionFact
 protected final static Map dbAdapters = new 
HashMap<>();
 protected final Map columnTypeMap = new HashMap<>();
 
+private static SimpleDateFormat TIME_TYPE_FORMAT = new 
SimpleDateFormat("HH:mm:ss.SSS");
+
 static {
 // Load the DatabaseAdapters
 ServiceLoader dbAdapterLoader = 
ServiceLoader.load(DatabaseAdapter.class);
@@ -275,14 +278,19 @@ public abstract class AbstractDatabaseFetchProcessor 
extends AbstractSessionFact
 break;
 
 case TIME:
-Date rawColTimeValue = resultSet.getDate(columnIndex);
-java.sql.Time colTimeValue = new 
java.sql.Time(rawColTimeValue.getTime());
-java.sql.Time maxTimeValue = null;
+// Compare milliseconds-since-epoch. Need getTimestamp() 
instead of getTime() since some databases
+// don't return milliseconds in the Time returned by getTime().
+Date colTimeValue = new 
Date(resultSet.getTimestamp(columnIndex).getTime());
+Date maxTimeValue = null;
 if (maxValueString != null) {
-maxTimeValue = java.sql.Time.valueOf(maxValueString);
+try {
+maxTimeValue = TIME_TYPE_FORMAT.parse(maxValueString);
+} catch (ParseException pe) {
+// Shouldn't happen, but just in case, leave the value 
as null so the new value will be stored
+}
 }
 if (maxTimeValue == null || colTimeValue.after(maxTimeValue)) {
-return colTimeValue.toString();
+return TIME_TYPE_FORMAT.format(colTimeValue);
 }
 break;
 
@@ -299,8 +307,7 @@ public abstract class AbstractDatabaseFetchProcessor 
extends AbstractSessionFact
 return oracleTimestampValue.toString();
 }
 } else {
-Timestamp rawColTimestampValue = 
resultSet.getTimestamp(columnIndex);
-java.sql.Timestamp colTimestampValue = new 
java.sql.Timestamp(rawColTimestampValue.getTime());
+Timestamp colTimestampValue = 
resultSet.getTimestamp(columnIndex);
 java.sql.Timestamp maxTimestampValue = null;
 if (maxValueString != null) {
 maxTimestampValue = 
java.sql.Timestamp.valueOf(maxValueString);


[2/2] nifi git commit: NIFI-2423: Make use of the SSLContextService to provide SSL information

2016-08-05 Thread ozhurakousky
NIFI-2423: Make use of the SSLContextService to provide SSL information

Signed-off-by: Oleg Zhurakousky 


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/dc9a46c6
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/dc9a46c6
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/dc9a46c6

Branch: refs/heads/0.x
Commit: dc9a46c637fb76f6b9fc8a841151c6be0c2ac836
Parents: 46a9bba
Author: Mark Payne 
Authored: Wed Aug 3 14:20:29 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Aug 5 14:21:43 2016 -0400

--
 .../nifi-kafka-pubsub-nar/pom.xml   |  5 ++
 .../nifi-kafka-pubsub-processors/pom.xml|  4 ++
 .../kafka/pubsub/AbstractKafkaProcessor.java| 59 +++-
 .../processors/kafka/pubsub/ConsumeKafka.java   |  1 -
 4 files changed, 43 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/dc9a46c6/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
index 45d6fb5..1254f77 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
@@ -31,5 +31,10 @@
 org.apache.nifi
 nifi-kafka-pubsub-processors
 
+
+org.apache.nifi
+nifi-standard-services-api-nar
+nar
+
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/dc9a46c6/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
index 3893c3a..7b869dc 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
@@ -35,6 +35,10 @@
 nifi-utils
 
 
+org.apache.nifi
+nifi-ssl-context-service-api
+
+
org.apache.kafka
kafka-clients
0.9.0.1

http://git-wip-us.apache.org/repos/asf/nifi/blob/dc9a46c6/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
index 04f9365..c2c2321 100644
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
@@ -43,6 +43,7 @@ import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.util.FormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -115,29 +116,12 @@ abstract class AbstractKafkaProcessor extends AbstractSessi
 .expressionLanguageSupported(true)
 .build();
 
-static final PropertyDescriptor SSL_KEY_PASSWORD = new 
PropertyDescriptor.Builder()
-.name("ssl.key.password")
-.displayName("SSL Key Password")
-.description("The password of the private key in the key store 
file. Corresponds to Kafka's 'ssl.key.password' property.")
+static final PropertyDescriptor SSL_CONTEXT_SERVICE = new 
PropertyDescriptor.Builder()
+.name("ssl.context.service")
+.displayName("SSL Context Service")
+.description("Specifies the SSL Context Service to use for 
communicating with Kafka.")
 .required(false)
-.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-.sensitive(true)
-.build();
-static final PropertyDescriptor SSL_KEYSTORE_PASSWORD = new 
PropertyDescriptor.Builder()
-.name("ssl.keystore.password")
-.displayName("SSK Keystore Password")
-   

[1/2] nifi git commit: NIFI-2322, NIFI-2423, NIFI-2412 Kafka improvements - Fixed KafkaConsumer's connection block when broker is not available - Fixed Serializer/Deserializer configs in both Consume/

2016-08-05 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x af61bbeac -> dc9a46c63


NIFI-2322, NIFI-2423, NIFI-2412 Kafka improvements
- Fixed KafkaConsumer's connection block when broker is not available
- Fixed Serializer/Deserializer configs in both Consume/Publish Kafka
- Added sensitive properties for SSL ket/trust stores

NIFI-2322 fixed tests


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/46a9bba3
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/46a9bba3
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/46a9bba3

Branch: refs/heads/0.x
Commit: 46a9bba31d650f9c45284ca97c9cf7b1150aaa22
Parents: af61bbe
Author: Oleg Zhurakousky 
Authored: Fri Jul 29 09:06:47 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Aug 5 14:21:24 2016 -0400

--
 .../kafka/pubsub/AbstractKafkaProcessor.java| 29 +++
 .../processors/kafka/pubsub/ConsumeKafka.java   | 52 ++--
 .../processors/kafka/pubsub/PublishKafka.java   |  8 ++-
 .../kafka/pubsub/ConsumeKafkaTest.java  |  2 +
 4 files changed, 86 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/46a9bba3/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
index 8bae304..04f9365 100644
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
@@ -115,6 +115,31 @@ abstract class AbstractKafkaProcessor 
extends AbstractSessi
 .expressionLanguageSupported(true)
 .build();
 
+static final PropertyDescriptor SSL_KEY_PASSWORD = new 
PropertyDescriptor.Builder()
+.name("ssl.key.password")
+.displayName("SSL Key Password")
+.description("The password of the private key in the key store 
file. Corresponds to Kafka's 'ssl.key.password' property.")
+.required(false)
+.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+.sensitive(true)
+.build();
+static final PropertyDescriptor SSL_KEYSTORE_PASSWORD = new 
PropertyDescriptor.Builder()
+.name("ssl.keystore.password")
+.displayName("SSK Keystore Password")
+.description("The store password for the key store file. 
Corresponds to Kafka's 'ssl.keystore.password' property.")
+.required(false)
+.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+.sensitive(true)
+.build();
+static final PropertyDescriptor SSL_TRUSTSTORE_PASSWORD = new 
PropertyDescriptor.Builder()
+.name("ssl.truststore.password")
+.displayName("SSL Truststore Password")
+.description("The password for the trust store file. Corresponds 
to Kafka's 'ssl.truststore.password' property.")
+.required(false)
+.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+.sensitive(true)
+.build();
+
 static final Builder MESSAGE_DEMARCATOR_BUILDER = new 
PropertyDescriptor.Builder()
 .name("message-demarcator")
 .displayName("Message Demarcator")
@@ -141,6 +166,10 @@ abstract class AbstractKafkaProcessor 
extends AbstractSessi
 SHARED_DESCRIPTORS.add(CLIENT_ID);
 SHARED_DESCRIPTORS.add(SECURITY_PROTOCOL);
 SHARED_DESCRIPTORS.add(KERBEROS_PRINCIPLE);
+SHARED_DESCRIPTORS.add(SSL_KEY_PASSWORD);
+SHARED_DESCRIPTORS.add(SSL_KEYSTORE_PASSWORD);
+SHARED_DESCRIPTORS.add(SSL_TRUSTSTORE_PASSWORD);
+
 SHARED_RELATIONSHIPS.add(REL_SUCCESS);
 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/46a9bba3/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
index 

[2/2] nifi git commit: NIFI-2322, NIFI-2423, NIFI-2412 Kafka improvements - Fixed KafkaConsumer's connection block when broker is not available - Fixed Serializer/Deserializer configs in both Consume/

2016-08-05 Thread ozhurakousky
NIFI-2322, NIFI-2423, NIFI-2412 Kafka improvements
- Fixed KafkaConsumer's connection block when broker is not available
- Fixed Serializer/Deserializer configs in both Consume/Publish Kafka
- Added sensitive properties for SSL ket/trust stores

NIFI-2322 fixed tests


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c39a127e
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c39a127e
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c39a127e

Branch: refs/heads/master
Commit: c39a127ec8e0d27efa6cedb2c5837310dccd3d6a
Parents: 8d380dc
Author: Oleg Zhurakousky 
Authored: Fri Jul 29 09:06:47 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Aug 5 14:14:38 2016 -0400

--
 .../kafka/pubsub/AbstractKafkaProcessor.java| 29 +++
 .../processors/kafka/pubsub/ConsumeKafka.java   | 52 ++--
 .../processors/kafka/pubsub/PublishKafka.java   |  8 ++-
 .../kafka/pubsub/ConsumeKafkaTest.java  |  2 +
 4 files changed, 86 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/c39a127e/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
index 8bae304..04f9365 100644
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
@@ -115,6 +115,31 @@ abstract class AbstractKafkaProcessor 
extends AbstractSessi
 .expressionLanguageSupported(true)
 .build();
 
+static final PropertyDescriptor SSL_KEY_PASSWORD = new 
PropertyDescriptor.Builder()
+.name("ssl.key.password")
+.displayName("SSL Key Password")
+.description("The password of the private key in the key store 
file. Corresponds to Kafka's 'ssl.key.password' property.")
+.required(false)
+.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+.sensitive(true)
+.build();
+static final PropertyDescriptor SSL_KEYSTORE_PASSWORD = new 
PropertyDescriptor.Builder()
+.name("ssl.keystore.password")
+.displayName("SSK Keystore Password")
+.description("The store password for the key store file. 
Corresponds to Kafka's 'ssl.keystore.password' property.")
+.required(false)
+.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+.sensitive(true)
+.build();
+static final PropertyDescriptor SSL_TRUSTSTORE_PASSWORD = new 
PropertyDescriptor.Builder()
+.name("ssl.truststore.password")
+.displayName("SSL Truststore Password")
+.description("The password for the trust store file. Corresponds 
to Kafka's 'ssl.truststore.password' property.")
+.required(false)
+.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+.sensitive(true)
+.build();
+
 static final Builder MESSAGE_DEMARCATOR_BUILDER = new 
PropertyDescriptor.Builder()
 .name("message-demarcator")
 .displayName("Message Demarcator")
@@ -141,6 +166,10 @@ abstract class AbstractKafkaProcessor 
extends AbstractSessi
 SHARED_DESCRIPTORS.add(CLIENT_ID);
 SHARED_DESCRIPTORS.add(SECURITY_PROTOCOL);
 SHARED_DESCRIPTORS.add(KERBEROS_PRINCIPLE);
+SHARED_DESCRIPTORS.add(SSL_KEY_PASSWORD);
+SHARED_DESCRIPTORS.add(SSL_KEYSTORE_PASSWORD);
+SHARED_DESCRIPTORS.add(SSL_TRUSTSTORE_PASSWORD);
+
 SHARED_RELATIONSHIPS.add(REL_SUCCESS);
 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/c39a127e/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java
index 2ed2db9..2bc1cfb 100644
--- 

[1/2] nifi git commit: NIFI-2423: Make use of the SSLContextService to provide SSL information

2016-08-05 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 8d380dcda -> 7ffa30d21


NIFI-2423: Make use of the SSLContextService to provide SSL information

Signed-off-by: Oleg Zhurakousky 


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/7ffa30d2
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/7ffa30d2
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/7ffa30d2

Branch: refs/heads/master
Commit: 7ffa30d21b2bc97ca280f881dc75a2f385deec1d
Parents: c39a127
Author: Mark Payne 
Authored: Wed Aug 3 14:20:29 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Fri Aug 5 14:14:38 2016 -0400

--
 .../nifi-kafka-pubsub-nar/pom.xml   |  5 ++
 .../nifi-kafka-pubsub-processors/pom.xml|  4 ++
 .../kafka/pubsub/AbstractKafkaProcessor.java| 59 +++-
 .../processors/kafka/pubsub/ConsumeKafka.java   |  1 -
 4 files changed, 43 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/7ffa30d2/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
index a9e3eb0..cb3be38 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-nar/pom.xml
@@ -31,5 +31,10 @@
 org.apache.nifi
 nifi-kafka-pubsub-processors
 
+
+org.apache.nifi
+nifi-standard-services-api-nar
+nar
+
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/7ffa30d2/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
index 53e309a..3ad8e37 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml
@@ -35,6 +35,10 @@
 nifi-utils
 
 
+org.apache.nifi
+nifi-ssl-context-service-api
+
+
org.apache.kafka
kafka-clients
0.9.0.1

http://git-wip-us.apache.org/repos/asf/nifi/blob/7ffa30d2/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
--
diff --git 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
index 04f9365..c2c2321 100644
--- 
a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
+++ 
b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java
@@ -43,6 +43,7 @@ import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.util.FormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -115,29 +116,12 @@ abstract class AbstractKafkaProcessor extends AbstractSessi
 .expressionLanguageSupported(true)
 .build();
 
-static final PropertyDescriptor SSL_KEY_PASSWORD = new 
PropertyDescriptor.Builder()
-.name("ssl.key.password")
-.displayName("SSL Key Password")
-.description("The password of the private key in the key store 
file. Corresponds to Kafka's 'ssl.key.password' property.")
+static final PropertyDescriptor SSL_CONTEXT_SERVICE = new 
PropertyDescriptor.Builder()
+.name("ssl.context.service")
+.displayName("SSL Context Service")
+.description("Specifies the SSL Context Service to use for 
communicating with Kafka.")
 .required(false)
-.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-.sensitive(true)
-.build();
-static final PropertyDescriptor SSL_KEYSTORE_PASSWORD = new 
PropertyDescriptor.Builder()
-

nifi git commit: NIFI-2160 fixed service startup ordering

2016-07-07 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 35a5667f3 -> 8dbb5d8aa


NIFI-2160 fixed service startup ordering

fixed imports


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/8dbb5d8a
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/8dbb5d8a
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/8dbb5d8a

Branch: refs/heads/0.x
Commit: 8dbb5d8aab1509bd27bc3ec31f28f9cfb19e6197
Parents: 35a5667
Author: Oleg Zhurakousky 
Authored: Tue Jul 5 17:47:51 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Jul 7 22:39:35 2016 -0400

--
 .../service/StandardControllerServiceNode.java  |  4 +-
 .../StandardControllerServiceProvider.java  | 27 
 .../TestStandardControllerServiceProvider.java  | 66 +++-
 3 files changed, 67 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/8dbb5d8a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index 2deb4ed..7a416af 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -115,7 +115,7 @@ public class StandardControllerServiceNode extends 
AbstractConfiguredComponent i
 
 @Override
 public List getRequiredControllerServices() {
-List requiredServices = new ArrayList<>();
+Set requiredServices = new HashSet<>();
 for (Entry pEntry : 
this.getProperties().entrySet()) {
 PropertyDescriptor descriptor = pEntry.getKey();
 if (descriptor.getControllerServiceDefinition() != null && 
descriptor.isRequired()) {
@@ -124,7 +124,7 @@ public class StandardControllerServiceNode extends 
AbstractConfiguredComponent i
 requiredServices.addAll(rNode.getRequiredControllerServices());
 }
 }
-return requiredServices;
+return new ArrayList<>(requiredServices);
 }
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/8dbb5d8a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 4b18751..58add5d 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -25,7 +25,6 @@ import java.lang.reflect.Proxy;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -313,17 +312,11 @@ public class StandardControllerServiceProvider implements 
ControllerServiceProvi
 }
 
 if (shouldStart) {
-List services = new 
ArrayList<>(serviceNodes);
-Collections.sort(services, new Comparator() 
{
-@Override
-public int compare(ControllerServiceNode s1, 
ControllerServiceNode s2) {
-return s2.getRequiredControllerServices().contains(s1) ? 
-1 : 1;
-}
-});
-
-for (ControllerServiceNode controllerServiceNode : services) {
+for (ControllerServiceNode controllerServiceNode : serviceNodes) {
 try {
-this.enableControllerService(controllerServiceNode);
+if (!controllerServiceNode.isActive()) {
+

nifi git commit: NIFI-2160 fixed service startup ordering (MASTER)

2016-07-07 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 6edfa634d -> 2ed1ab763


NIFI-2160 fixed service startup ordering (MASTER)

This closes #606


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/2ed1ab76
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/2ed1ab76
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/2ed1ab76

Branch: refs/heads/master
Commit: 2ed1ab7630a91a0f966954f9821048693fe65598
Parents: 6edfa63
Author: Oleg Zhurakousky 
Authored: Tue Jul 5 20:08:34 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Jul 7 22:37:20 2016 -0400

--
 .../service/StandardControllerServiceNode.java  |  6 +-
 .../StandardControllerServiceProvider.java  | 28 ---
 .../TestStandardControllerServiceProvider.java  | 85 +++-
 3 files changed, 82 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/2ed1ab76/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index 05dbc2d..7856dcd 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -22,8 +22,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -170,7 +170,7 @@ public class StandardControllerServiceNode extends 
AbstractConfiguredComponent i
 
 @Override
 public List getRequiredControllerServices() {
-List requiredServices = new ArrayList<>();
+Set requiredServices = new HashSet<>();
 for (Entry pEntry : 
this.getProperties().entrySet()) {
 PropertyDescriptor descriptor = pEntry.getKey();
 if (descriptor.getControllerServiceDefinition() != null && 
descriptor.isRequired()) {
@@ -179,7 +179,7 @@ public class StandardControllerServiceNode extends 
AbstractConfiguredComponent i
 requiredServices.addAll(rNode.getRequiredControllerServices());
 }
 }
-return requiredServices;
+return new ArrayList<>(requiredServices);
 }
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/2ed1ab76/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 3861355..22ee5cf 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -25,7 +25,6 @@ import java.lang.reflect.Proxy;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -55,7 +54,6 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.processor.StandardValidationContextFactory;
-
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.util.ObjectHolder;
@@ -385,17 +383,11 @@ public class StandardControllerServiceProvider implements 
ControllerServiceProvi
 }
 
 if (shouldStart) {
-  

nifi git commit: NIFI-2113 Updating storm dependency to 1.0.1 and fixing package names

2016-07-06 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master 293dc2993 -> 9d273b1e2


NIFI-2113 Updating storm dependency to 1.0.1 and fixing package names

This closes #578


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/9d273b1e
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/9d273b1e
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/9d273b1e

Branch: refs/heads/master
Commit: 9d273b1e2f25e76d901fe143da97f86d575f845b
Parents: 293dc29
Author: Bryan Bende 
Authored: Fri Jun 24 14:46:44 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Wed Jul 6 13:03:08 2016 -0400

--
 nifi-external/nifi-storm-spout/pom.xml   |  2 +-
 .../main/java/org/apache/nifi/storm/NiFiBolt.java| 15 ---
 .../org/apache/nifi/storm/NiFiDataPacketBuilder.java |  2 +-
 .../main/java/org/apache/nifi/storm/NiFiSpout.java   | 14 +++---
 .../java/org/apache/nifi/storm/MockTupleHelpers.java |  4 ++--
 .../org/apache/nifi/storm/NiFiStormTopology.java | 10 +-
 .../java/org/apache/nifi/storm/TestNiFiBolt.java |  6 +++---
 7 files changed, 27 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/9d273b1e/nifi-external/nifi-storm-spout/pom.xml
--
diff --git a/nifi-external/nifi-storm-spout/pom.xml 
b/nifi-external/nifi-storm-spout/pom.xml
index 301b41f..1fbf70d 100644
--- a/nifi-external/nifi-storm-spout/pom.xml
+++ b/nifi-external/nifi-storm-spout/pom.xml
@@ -27,7 +27,7 @@
 
 org.apache.storm
 storm-core
-0.10.0
+1.0.1
 provided
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/9d273b1e/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiBolt.java
--
diff --git 
a/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiBolt.java
 
b/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiBolt.java
index 64cd0de..733cb69 100644
--- 
a/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiBolt.java
+++ 
b/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiBolt.java
@@ -16,18 +16,19 @@
  */
 package org.apache.nifi.storm;
 
-import backtype.storm.Config;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.utils.TupleUtils;
+
 import org.apache.commons.lang3.Validate;
 import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.SiteToSiteClient;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+import org.apache.storm.Config;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.utils.TupleUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/9d273b1e/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiDataPacketBuilder.java
--
diff --git 
a/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiDataPacketBuilder.java
 
b/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiDataPacketBuilder.java
index fa2e20e..c3653bf 100644
--- 
a/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiDataPacketBuilder.java
+++ 
b/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiDataPacketBuilder.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.storm;
 
-import backtype.storm.tuple.Tuple;
+import org.apache.storm.tuple.Tuple;
 
 /**
  * Converts a Tuple into a NiFiDataPacket.

http://git-wip-us.apache.org/repos/asf/nifi/blob/9d273b1e/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiSpout.java
--
diff --git 
a/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiSpout.java
 
b/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiSpout.java
index 2cb0b66..791fe1d 100644
--- 
a/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiSpout.java
+++ 
b/nifi-external/nifi-storm-spout/src/main/java/org/apache/nifi/storm/NiFiSpout.java
@@ -16,19 +16,19 @@
  */
 package org.apache.nifi.storm;
 

nifi git commit: NIFI-2097 Changing RemoteProcessGroupStatusTest.java to use a random open port instead of a statically defined one

2016-06-23 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 6e7b67e84 -> 03c937623


NIFI-2097 Changing RemoteProcessGroupStatusTest.java to use a random open port 
instead of a statically defined one

This closes #569


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/03c93762
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/03c93762
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/03c93762

Branch: refs/heads/0.x
Commit: 03c9376233e4669c32bf4b62e3f33368b1184619
Parents: 6e7b67e
Author: jpercivall 
Authored: Thu Jun 23 13:24:15 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Jun 23 14:25:11 2016 -0400

--
 .../apache/nifi/remote/RemoteProcessGroupStatusTest.java  | 10 +-
 1 file changed, 9 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/03c93762/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/remote/RemoteProcessGroupStatusTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/remote/RemoteProcessGroupStatusTest.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/remote/RemoteProcessGroupStatusTest.java
index 56f7074..98e9eb9 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/remote/RemoteProcessGroupStatusTest.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/remote/RemoteProcessGroupStatusTest.java
@@ -19,7 +19,9 @@ package org.apache.nifi.remote;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
+import java.net.ServerSocket;
 import java.net.URL;
 import java.util.Collections;
 
@@ -58,12 +60,18 @@ public class RemoteProcessGroupStatusTest {
 }
 
 @Before
-public void before() {
+public void before() throws IOException {
+int port;
+try (ServerSocket socket = new ServerSocket(0)) {
+socket.setReuseAddress(true);
+port = socket.getLocalPort();
+}
 NiFiProperties properties = NiFiProperties.getInstance();
 
properties.setProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, 
MockProvenanceEventRepository.class.getName());
 properties.setProperty(NiFiProperties.STATE_MANAGEMENT_CONFIG_FILE, 
"src/test/resources/state-management.xml");
 
properties.setProperty(NiFiProperties.STATE_MANAGEMENT_LOCAL_PROVIDER_ID, 
"local-provider");
 properties.setProperty(NiFiProperties.REMOTE_INPUT_HOST, "localhost");
+properties.setProperty(NiFiProperties.REMOTE_INPUT_PORT, 
String.valueOf(port));
 properties.setProperty("nifi.remote.input.secure", "false");
 
 RingBufferEventRepository repository = new 
RingBufferEventRepository(1);



nifi git commit: NIFI-2092 Fixing unit tests which try to clean up directories before stopping the FlowController that has locks opened.

2016-06-23 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 7b7788ef7 -> 6e7b67e84


NIFI-2092 Fixing unit tests which try to clean up directories before stopping 
the FlowController that has locks opened.

This closes #568


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6e7b67e8
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6e7b67e8
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6e7b67e8

Branch: refs/heads/0.x
Commit: 6e7b67e8480d5c2772eac90a4a8aae32704b5fb0
Parents: 7b7788e
Author: jpercivall 
Authored: Thu Jun 23 12:42:16 2016 -0400
Committer: Oleg Zhurakousky 
Committed: Thu Jun 23 14:19:02 2016 -0400

--
 .../apache/nifi/controller/scheduling/TestProcessorLifecycle.java  | 2 +-
 .../test/java/org/apache/nifi/controller/MonitorMemoryTest.java| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/6e7b67e8/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
--
diff --git 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
index df52516..f0756a8 100644
--- 
a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
+++ 
b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
@@ -90,9 +90,9 @@ public class TestProcessorLifecycle {
 
 @After
 public void after() throws Exception {
+fc.shutdown(true);
 FileUtils.deleteDirectory(new File("./target/test-repo"));
 FileUtils.deleteDirectory(new File("./target/content_repository"));
-fc.shutdown(true);
 }
 
 @Test

http://git-wip-us.apache.org/repos/asf/nifi/blob/6e7b67e8/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
index b81d952..54f6aab 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
@@ -51,9 +51,9 @@ public class MonitorMemoryTest {
 
 @After
 public void after() throws Exception {
+fc.shutdown(true);
 FileUtils.deleteDirectory(new File("./target/test-repo"));
 FileUtils.deleteDirectory(new File("./target/content_repository"));
-fc.shutdown(true);
 }
 
 @Test(expected = IllegalStateException.class)



nifi git commit: NIFI-2066 dynamic port allocation in SNMP processors unit tests This closes #552

2016-06-21 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 378e19534 -> f571379fe


NIFI-2066 dynamic port allocation in SNMP processors unit tests
This closes #552


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f571379f
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f571379f
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f571379f

Branch: refs/heads/0.x
Commit: f571379fe667c968849a0e4136dc38923cf4bbb9
Parents: 378e195
Author: Pierre Villard 
Authored: Tue Jun 21 19:16:43 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Tue Jun 21 14:47:44 2016 -0400

--
 .../nifi/snmp/processors/GetSNMPTest.java   | 50 +++-
 .../nifi/snmp/processors/SNMPGetTest.java   |  8 ++--
 .../nifi/snmp/processors/SNMPTestUtil.java  | 44 +
 .../nifi/snmp/processors/SNMPUtilsTest.java |  4 +-
 .../nifi/snmp/processors/SNMPWalkTest.java  |  8 ++--
 .../nifi/snmp/processors/SetSNMPTest.java   | 32 ++---
 .../nifi/snmp/processors/TestSnmpAgentV1.java   | 10 +++-
 .../nifi/snmp/processors/TestSnmpAgentV2c.java  |  9 +++-
 .../nifi/snmp/processors/TestSnmpAgentV3.java   |  2 +-
 .../nifi/snmp/processors/WalkSNMPTest.java  |  6 +--
 10 files changed, 117 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/f571379f/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
index aed6d9c..1d40e25 100644
--- 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
@@ -70,7 +70,7 @@ public class GetSNMPTest {
  */
 @BeforeClass
 public static void setUp() throws Exception {
-agentv2c = new TestSnmpAgentV2c("0.0.0.0/2001");
+agentv2c = new TestSnmpAgentV2c("0.0.0.0");
 agentv2c.start();
 agentv2c.unregisterManagedObject(agentv2c.getSnmpv2MIB());
 agentv2c.registerManagedObject(
@@ -82,7 +82,7 @@ public class GetSNMPTest {
 MOAccessImpl.ACCESS_WRITE_ONLY,
 new Integer32(writeOnlyValue)));
 
-agentv1 = new TestSnmpAgentV1("0.0.0.0/2002");
+agentv1 = new TestSnmpAgentV1("0.0.0.0");
 agentv1.start();
 agentv1.unregisterManagedObject(agentv1.getSnmpv2MIB());
 agentv1.registerManagedObject(
@@ -111,7 +111,7 @@ public class GetSNMPTest {
 @Test
 public void validateSuccessfullSnmpSetGetv2c() throws Exception {
 Snmp snmp = SNMPUtilsTest.createSnmp();
-CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/2001", SnmpConstants.version2c);
+CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/" + agentv2c.getPort(), SnmpConstants.version2c);
 
 try (SNMPSetter setter = new SNMPSetter(snmp, target)) {
 PDU pdu = new PDU();
@@ -128,7 +128,7 @@ public class GetSNMPTest {
 
 runner.setProperty(GetSNMP.OID, sysDescr.toString());
 runner.setProperty(GetSNMP.HOST, "127.0.0.1");
-runner.setProperty(GetSNMP.PORT, "2001");
+runner.setProperty(GetSNMP.PORT, 
String.valueOf(agentv2c.getPort()));
 runner.setProperty(GetSNMP.SNMP_COMMUNITY, "public");
 runner.setProperty(GetSNMP.SNMP_VERSION, "SNMPv2c");
 
@@ -152,7 +152,7 @@ public class GetSNMPTest {
 @Test
 public void validateSuccessfullSnmpSetGetv1() throws Exception {
 Snmp snmp = SNMPUtilsTest.createSnmp();
-CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/2002", SnmpConstants.version1);
+CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/" + agentv1.getPort(), SnmpConstants.version1);
 
 try (SNMPSetter setter = new SNMPSetter(snmp, target)) {
 PDU pdu = new PDU();
@@ -168,7 +168,7 @@ public class GetSNMPTest {
 TestRunner runner = TestRunners.newTestRunner(pubProc);
 runner.setProperty(GetSNMP.OID, sysDescr.toString());
 runner.setProperty(GetSNMP.HOST, "127.0.0.1");
-runner.setProperty(GetSNMP.PORT, "2002");
+runner.setProperty(GetSNMP.PORT, 
String.valueOf(agentv1.getPort()));
 

nifi git commit: NIFI-2066 dynamic port allocation in SNMP processors unit tests This closes #552

2016-06-21 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master ef192cc85 -> 6f3abdbcb


NIFI-2066 dynamic port allocation in SNMP processors unit tests
This closes #552


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6f3abdbc
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6f3abdbc
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6f3abdbc

Branch: refs/heads/master
Commit: 6f3abdbcbd4aa0d36428c623c83b0c1e18481e39
Parents: ef192cc
Author: Pierre Villard 
Authored: Tue Jun 21 19:16:43 2016 +0200
Committer: Oleg Zhurakousky 
Committed: Tue Jun 21 14:46:21 2016 -0400

--
 .../nifi/snmp/processors/GetSNMPTest.java   | 50 +++-
 .../nifi/snmp/processors/SNMPGetTest.java   |  8 ++--
 .../nifi/snmp/processors/SNMPTestUtil.java  | 44 +
 .../nifi/snmp/processors/SNMPUtilsTest.java |  4 +-
 .../nifi/snmp/processors/SNMPWalkTest.java  |  8 ++--
 .../nifi/snmp/processors/SetSNMPTest.java   | 32 ++---
 .../nifi/snmp/processors/TestSnmpAgentV1.java   | 10 +++-
 .../nifi/snmp/processors/TestSnmpAgentV2c.java  |  9 +++-
 .../nifi/snmp/processors/TestSnmpAgentV3.java   |  2 +-
 .../nifi/snmp/processors/WalkSNMPTest.java  |  6 +--
 10 files changed, 117 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/6f3abdbc/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
index aed6d9c..1d40e25 100644
--- 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
@@ -70,7 +70,7 @@ public class GetSNMPTest {
  */
 @BeforeClass
 public static void setUp() throws Exception {
-agentv2c = new TestSnmpAgentV2c("0.0.0.0/2001");
+agentv2c = new TestSnmpAgentV2c("0.0.0.0");
 agentv2c.start();
 agentv2c.unregisterManagedObject(agentv2c.getSnmpv2MIB());
 agentv2c.registerManagedObject(
@@ -82,7 +82,7 @@ public class GetSNMPTest {
 MOAccessImpl.ACCESS_WRITE_ONLY,
 new Integer32(writeOnlyValue)));
 
-agentv1 = new TestSnmpAgentV1("0.0.0.0/2002");
+agentv1 = new TestSnmpAgentV1("0.0.0.0");
 agentv1.start();
 agentv1.unregisterManagedObject(agentv1.getSnmpv2MIB());
 agentv1.registerManagedObject(
@@ -111,7 +111,7 @@ public class GetSNMPTest {
 @Test
 public void validateSuccessfullSnmpSetGetv2c() throws Exception {
 Snmp snmp = SNMPUtilsTest.createSnmp();
-CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/2001", SnmpConstants.version2c);
+CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/" + agentv2c.getPort(), SnmpConstants.version2c);
 
 try (SNMPSetter setter = new SNMPSetter(snmp, target)) {
 PDU pdu = new PDU();
@@ -128,7 +128,7 @@ public class GetSNMPTest {
 
 runner.setProperty(GetSNMP.OID, sysDescr.toString());
 runner.setProperty(GetSNMP.HOST, "127.0.0.1");
-runner.setProperty(GetSNMP.PORT, "2001");
+runner.setProperty(GetSNMP.PORT, 
String.valueOf(agentv2c.getPort()));
 runner.setProperty(GetSNMP.SNMP_COMMUNITY, "public");
 runner.setProperty(GetSNMP.SNMP_VERSION, "SNMPv2c");
 
@@ -152,7 +152,7 @@ public class GetSNMPTest {
 @Test
 public void validateSuccessfullSnmpSetGetv1() throws Exception {
 Snmp snmp = SNMPUtilsTest.createSnmp();
-CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/2002", SnmpConstants.version1);
+CommunityTarget target = SNMPUtilsTest.createCommTarget("public", 
"127.0.0.1/" + agentv1.getPort(), SnmpConstants.version1);
 
 try (SNMPSetter setter = new SNMPSetter(snmp, target)) {
 PDU pdu = new PDU();
@@ -168,7 +168,7 @@ public class GetSNMPTest {
 TestRunner runner = TestRunners.newTestRunner(pubProc);
 runner.setProperty(GetSNMP.OID, sysDescr.toString());
 runner.setProperty(GetSNMP.HOST, "127.0.0.1");
-runner.setProperty(GetSNMP.PORT, "2002");
+runner.setProperty(GetSNMP.PORT, 
String.valueOf(agentv1.getPort()));
 

[2/2] nifi git commit: NIFI-1537 Added SNMP processors This closes #257

2016-06-19 Thread ozhurakousky
NIFI-1537 Added SNMP processors
This closes #257


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/a6e97740
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/a6e97740
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/a6e97740

Branch: refs/heads/0.x
Commit: a6e97740c102e22917f4ebb3e1572e6918a3b55a
Parents: 1398312
Author: Pierre Villard 
Authored: Sun Feb 28 10:12:02 2016 +0100
Committer: Oleg Zhurakousky 
Committed: Sun Jun 19 15:00:33 2016 -0400

--
 nifi-assembly/pom.xml   |   5 +
 .../nifi-snmp-bundle/nifi-snmp-nar/pom.xml  |  35 ++
 .../src/main/resources/META-INF/LICENSE | 203 +
 .../src/main/resources/META-INF/NOTICE  |  24 ++
 .../nifi-snmp-processors/pom.xml|  81 
 .../snmp/processors/AbstractSNMPProcessor.java  | 426 +++
 .../apache/nifi/snmp/processors/GetSNMP.java| 177 
 .../apache/nifi/snmp/processors/SNMPGetter.java | 100 +
 .../apache/nifi/snmp/processors/SNMPSetter.java |  56 +++
 .../apache/nifi/snmp/processors/SNMPUtils.java  | 268 
 .../apache/nifi/snmp/processors/SNMPWorker.java |  74 
 .../apache/nifi/snmp/processors/SetSNMP.java| 224 ++
 .../org.apache.nifi.processor.Processor |  16 +
 .../additionalDetails.html  |  70 +++
 .../additionalDetails.html  |  58 +++
 .../nifi/snmp/processors/GetSNMPTest.java   | 385 +
 .../nifi/snmp/processors/SNMPGetTest.java   | 123 ++
 .../nifi/snmp/processors/SNMPUtilsTest.java | 134 ++
 .../nifi/snmp/processors/SNMPWalkTest.java  | 121 ++
 .../nifi/snmp/processors/SetSNMPTest.java   | 332 +++
 .../nifi/snmp/processors/TestSnmpAgentV1.java   | 212 +
 .../nifi/snmp/processors/TestSnmpAgentV2c.java  | 212 +
 .../nifi/snmp/processors/TestSnmpAgentV3.java   | 255 +++
 .../nifi/snmp/processors/WalkSNMPTest.java  | 117 +
 .../src/test/resources/log4j.properties |  20 +
 nifi-nar-bundles/nifi-snmp-bundle/pom.xml   |  39 ++
 nifi-nar-bundles/pom.xml|   1 +
 pom.xml |   6 +
 28 files changed, 3774 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/a6e97740/nifi-assembly/pom.xml
--
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index 9d68ba2..f50f6e8 100644
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -336,6 +336,11 @@ language governing permissions and limitations under the 
License. -->
 org.apache.nifi
 nifi-mqtt-nar
 nar
+   
+
+org.apache.nifi
+nifi-snmp-nar
+nar
 
 
 org.apache.nifi

http://git-wip-us.apache.org/repos/asf/nifi/blob/a6e97740/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml
new file mode 100644
index 000..38e5f27
--- /dev/null
+++ b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml
@@ -0,0 +1,35 @@
+
+
+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;>
+4.0.0
+
+org.apache.nifi
+nifi-snmp-bundle
+1.0.0-SNAPSHOT
+
+nifi-snmp-nar
+nar
+
+true
+true
+
+
+
+org.apache.nifi
+nifi-snmp-processors
+
+
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/a6e97740/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 000..6b0b127
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,203 @@
+
+ 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.
+
+  

[1/2] nifi git commit: NIFI-1537 Added SNMP processors This closes #257

2016-06-19 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x 139831241 -> a6e97740c


http://git-wip-us.apache.org/repos/asf/nifi/blob/a6e97740/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
new file mode 100644
index 000..3e0fb25
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
@@ -0,0 +1,58 @@
+
+
+
+
+
+SetSNMP
+
+
+
+
+Summary
+
+This processor sends SNMP set requests to a SNMP agent in order to update 
information associated to a given OID. This processor supports 
+SNMPv1, SNMPv2c and SNMPv3. The component is based on http://www.snmp4j.org/;>SNMP4J.
+
+
+The processor constructs SNMP Set requests by extracting information from 
FlowFile attributes. The processor is looking for attributes
+prefixed with snmp$. If such an attribute is found, the attribute 
name is splitted using the $ character. The second element must
+respect the OID format to be considered as a valid OID. If there is a 
third element, it must represents the SMI Syntax integer value of
+the type of data associated to the given OID to allow a correct 
conversion. If there is no third element, the value is considered as a String
+and the value will be sent as an OctetString object.
+
+Configuration Details
+
+At the time of writing this document it only defines the essential 
configuration properties which are suitable for most cases. 
+Other properties will be defined later as this component progresses.
+Configuring SetSNMP:
+
+
+Host - [REQUIRED] the name of the host where the SNMP agent is 
running. (default is localhost)
+Port - [REQUIRED] the port number on which the SNMP agent is 
listening. (default is 161)
+Version - [REQUIRED] the SNMP version to use (SNMPv1 [default], 
or SNMPv2c, or SNMPv3)
+Community - [REQUIRED if SNMPv1 or SNMPv2c] the SNMP community 
to use. (default is public)
+Security Level - [REQUIRED if SNMPv3] the security level to 
use. (default is authPriv)
+Security Name - [REQUIRED if SNMPv3] the security name (user 
name) to use.
+Authentication Protocol - [REQUIRED if SNMPv3 and authNoPriv or 
authPriv] the authentication protocol to use.
+Authentication Password - [REQUIRED if SNMPv3 and authNoPriv or 
authPriv] the authentication password to use.
+Private Protocol - [REQUIRED if SNMPv3 and authPriv] the 
private protocol to use.
+Private Password - [REQUIRED if SNMPv3 and authPriv] the 
private password to use.
+Retries - [REQUIRED] Sets the number of retries to be performed 
before a request is timed out. (default is 0)
+Timeout - [REQUIRED] Sets  timeout in milliseconds before a 
confirmed request is resent or timed out. (default is 5000)
+
+
+
+
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a6e97740/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
new file mode 100644
index 000..aed6d9c
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
@@ -0,0 +1,385 @@
+/*
+ * 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.snmp.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static 

[2/2] nifi git commit: NIFI-1537 Added SNMP processors This closes #257

2016-06-19 Thread ozhurakousky
NIFI-1537 Added SNMP processors
This closes #257


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/86bba1b2
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/86bba1b2
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/86bba1b2

Branch: refs/heads/master
Commit: 86bba1b20241fb8f21eccf08f5a56b2fda12e740
Parents: b1b7e35
Author: Pierre Villard 
Authored: Sun Feb 28 10:12:02 2016 +0100
Committer: Oleg Zhurakousky 
Committed: Sun Jun 19 14:58:41 2016 -0400

--
 nifi-assembly/pom.xml   |   5 +
 .../nifi-snmp-bundle/nifi-snmp-nar/pom.xml  |  35 ++
 .../src/main/resources/META-INF/LICENSE | 203 +
 .../src/main/resources/META-INF/NOTICE  |  24 ++
 .../nifi-snmp-processors/pom.xml|  81 
 .../snmp/processors/AbstractSNMPProcessor.java  | 426 +++
 .../apache/nifi/snmp/processors/GetSNMP.java| 177 
 .../apache/nifi/snmp/processors/SNMPGetter.java | 100 +
 .../apache/nifi/snmp/processors/SNMPSetter.java |  56 +++
 .../apache/nifi/snmp/processors/SNMPUtils.java  | 268 
 .../apache/nifi/snmp/processors/SNMPWorker.java |  74 
 .../apache/nifi/snmp/processors/SetSNMP.java| 224 ++
 .../org.apache.nifi.processor.Processor |  16 +
 .../additionalDetails.html  |  70 +++
 .../additionalDetails.html  |  58 +++
 .../nifi/snmp/processors/GetSNMPTest.java   | 385 +
 .../nifi/snmp/processors/SNMPGetTest.java   | 123 ++
 .../nifi/snmp/processors/SNMPUtilsTest.java | 134 ++
 .../nifi/snmp/processors/SNMPWalkTest.java  | 121 ++
 .../nifi/snmp/processors/SetSNMPTest.java   | 332 +++
 .../nifi/snmp/processors/TestSnmpAgentV1.java   | 212 +
 .../nifi/snmp/processors/TestSnmpAgentV2c.java  | 212 +
 .../nifi/snmp/processors/TestSnmpAgentV3.java   | 255 +++
 .../nifi/snmp/processors/WalkSNMPTest.java  | 117 +
 .../src/test/resources/log4j.properties |  20 +
 nifi-nar-bundles/nifi-snmp-bundle/pom.xml   |  39 ++
 nifi-nar-bundles/pom.xml|   1 +
 pom.xml |   6 +
 28 files changed, 3774 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/86bba1b2/nifi-assembly/pom.xml
--
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index 888533b..0ae6237 100644
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -331,6 +331,11 @@ language governing permissions and limitations under the 
License. -->
 org.apache.nifi
 nifi-mqtt-nar
 nar
+   
+
+org.apache.nifi
+nifi-snmp-nar
+nar
 
 
 org.apache.nifi

http://git-wip-us.apache.org/repos/asf/nifi/blob/86bba1b2/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml
--
diff --git a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml
new file mode 100644
index 000..38e5f27
--- /dev/null
+++ b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/pom.xml
@@ -0,0 +1,35 @@
+
+
+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;>
+4.0.0
+
+org.apache.nifi
+nifi-snmp-bundle
+1.0.0-SNAPSHOT
+
+nifi-snmp-nar
+nar
+
+true
+true
+
+
+
+org.apache.nifi
+nifi-snmp-processors
+
+
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/86bba1b2/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 000..6b0b127
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,203 @@
+
+ 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.
+
+   

[1/2] nifi git commit: NIFI-1537 Added SNMP processors This closes #257

2016-06-19 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/master b1b7e35be -> 86bba1b20


http://git-wip-us.apache.org/repos/asf/nifi/blob/86bba1b2/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
new file mode 100644
index 000..3e0fb25
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/main/resources/docs/org.apache.nifi.snmp.processors.SetSNMP/additionalDetails.html
@@ -0,0 +1,58 @@
+
+
+
+
+
+SetSNMP
+
+
+
+
+Summary
+
+This processor sends SNMP set requests to a SNMP agent in order to update 
information associated to a given OID. This processor supports 
+SNMPv1, SNMPv2c and SNMPv3. The component is based on http://www.snmp4j.org/;>SNMP4J.
+
+
+The processor constructs SNMP Set requests by extracting information from 
FlowFile attributes. The processor is looking for attributes
+prefixed with snmp$. If such an attribute is found, the attribute 
name is splitted using the $ character. The second element must
+respect the OID format to be considered as a valid OID. If there is a 
third element, it must represents the SMI Syntax integer value of
+the type of data associated to the given OID to allow a correct 
conversion. If there is no third element, the value is considered as a String
+and the value will be sent as an OctetString object.
+
+Configuration Details
+
+At the time of writing this document it only defines the essential 
configuration properties which are suitable for most cases. 
+Other properties will be defined later as this component progresses.
+Configuring SetSNMP:
+
+
+Host - [REQUIRED] the name of the host where the SNMP agent is 
running. (default is localhost)
+Port - [REQUIRED] the port number on which the SNMP agent is 
listening. (default is 161)
+Version - [REQUIRED] the SNMP version to use (SNMPv1 [default], 
or SNMPv2c, or SNMPv3)
+Community - [REQUIRED if SNMPv1 or SNMPv2c] the SNMP community 
to use. (default is public)
+Security Level - [REQUIRED if SNMPv3] the security level to 
use. (default is authPriv)
+Security Name - [REQUIRED if SNMPv3] the security name (user 
name) to use.
+Authentication Protocol - [REQUIRED if SNMPv3 and authNoPriv or 
authPriv] the authentication protocol to use.
+Authentication Password - [REQUIRED if SNMPv3 and authNoPriv or 
authPriv] the authentication password to use.
+Private Protocol - [REQUIRED if SNMPv3 and authPriv] the 
private protocol to use.
+Private Password - [REQUIRED if SNMPv3 and authPriv] the 
private password to use.
+Retries - [REQUIRED] Sets the number of retries to be performed 
before a request is timed out. (default is 0)
+Timeout - [REQUIRED] Sets  timeout in milliseconds before a 
confirmed request is resent or timed out. (default is 5000)
+
+
+
+
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/86bba1b2/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
--
diff --git 
a/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
new file mode 100644
index 000..aed6d9c
--- /dev/null
+++ 
b/nifi-nar-bundles/nifi-snmp-bundle/nifi-snmp-processors/src/test/java/org/apache/nifi/snmp/processors/GetSNMPTest.java
@@ -0,0 +1,385 @@
+/*
+ * 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.snmp.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static 

nifi git commit: NIFI-1955: Deprecate IntegerHolder, LongHolder, BooleanHolder

2016-06-17 Thread ozhurakousky
Repository: nifi
Updated Branches:
  refs/heads/0.x ff4b91e70 -> f8ae10f75


NIFI-1955: Deprecate IntegerHolder, LongHolder, BooleanHolder

This closes #537


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f8ae10f7
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f8ae10f7
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f8ae10f7

Branch: refs/heads/0.x
Commit: f8ae10f75aa6308c5a5c70523e2362248a4b0023
Parents: ff4b91e
Author: Joshi 
Authored: Thu Jun 16 19:49:00 2016 -0700
Committer: Oleg Zhurakousky 
Committed: Fri Jun 17 12:34:54 2016 -0400

--
 .../java/org/apache/nifi/util/BooleanHolder.java | 12 
 .../java/org/apache/nifi/util/IntegerHolder.java | 15 ---
 .../main/java/org/apache/nifi/util/LongHolder.java   |  9 -
 3 files changed, 32 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/nifi/blob/f8ae10f7/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
--
diff --git 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
index 92061e0..8283389 100644
--- 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
+++ 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
@@ -16,6 +16,18 @@
  */
 package org.apache.nifi.util;
 
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * @deprecated As of release 1.0.1. Please use {@link AtomicBoolean}
+ *
+ * Wraps an Boolean value so that it can be declared final and 
still be accessed from inner classes;
+ * the functionality is similar to that of an AtomicBoolean, but operations on 
this class
+ * are not atomic. This results in greater performance when the atomicity is 
not needed.
+ *
+ */
+
+@Deprecated
 public class BooleanHolder extends ObjectHolder {
 
 public BooleanHolder(final boolean initialValue) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8ae10f7/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
--
diff --git 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
index 213bbc0..8abfdb1 100644
--- 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
+++ 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
@@ -16,6 +16,18 @@
  */
 package org.apache.nifi.util;
 
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * @deprecated As of release 1.0.1. Please use {@link AtomicInteger}
+ *
+ * Wraps an Integer value so that it can be declared final and 
still be accessed from inner classes;
+ * the functionality is similar to that of an AtomicInteger, but operations on 
this class
+ * are not atomic. This results in greater performance when the atomicity is 
not needed.
+ *
+ */
+
+@Deprecated
 public class IntegerHolder extends ObjectHolder {
 
 public IntegerHolder(final int initialValue) {
@@ -48,7 +60,4 @@ public class IntegerHolder extends ObjectHolder {
 return addAndGet(-1);
 }
 
-public int getAndDecrement() {
-return getAndAdd(-1);
-}
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8ae10f7/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
--
diff --git 
a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java 
b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
index fa2d063..723463a 100644
--- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
@@ -16,10 +16,17 @@
  */
 package org.apache.nifi.util;
 
+import java.util.concurrent.atomic.AtomicLong;
+
 /**
- * Wraps a Long value so that it can be declared final and still 
be accessed from which inner classes; the functionality is similar to that of 
an AtomicLong, but operations on this class
+ * @deprecated As of release 1.0.1. Please use {@link AtomicLong}
+ *
+ * Wraps a Long value so that it can be declared final and still 
be accessed from inner classes;
+ * the functionality is similar to that of an AtomicLong, but operations on 
this class
  * are not atomic. This results in greater performance when the atomicity is 
not needed.
  */
+
+@Deprecated
 public class LongHolder extends ObjectHolder {
 
 public LongHolder(final long initialValue) {



  1   2   >