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
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 PayneAuthored: 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
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 MapforkEventBuilders = 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
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 MapcomponentIdMap; +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
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
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
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 PayneAuthored: 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
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 MapcomponentIdMap; +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
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
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
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
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
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(Mapevents); + +/** + * 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
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 ZhurakouskyAuthored: 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
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 Maptransform(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
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 ZhurakouskyAuthored: 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
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 VillardAuthored: 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
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 VillardAuthored: 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
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 PiriAuthored: 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
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 KawamuraAuthored: 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
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 KawamuraAuthored: 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
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 ZhurakouskyAuthored: 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
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 PsaltisAuthored: 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.
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 KawamuraAuthored: 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
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 PayneAuthored: 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
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 GilmanAuthored: 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.
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.
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 Mapattributes = 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.
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 PayneAuthored: 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.
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.
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 Mapattrs = 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.
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.
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
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 StorckAuthored: 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
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: jpercivallAuthored: 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.
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 KawamuraAuthored: 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.
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.
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.
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 KawamuraAuthored: 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
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 PayneAuthored: 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
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 BendeAuthored: 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
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 GelhausenAuthored: 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
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 PayneAuthored: 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
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 StorckAuthored: 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
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 BurgessAuthored: 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()
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 BendeAuthored: 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
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
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
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 BendeAuthored: 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
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 ZhurakouskyAuthored: 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
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 VillardAuthored: 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
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 WingAuthored: 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
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 BurgessAuthored: 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
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 VillardAuthored: 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
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 NAuthored: 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
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: patrickerAuthored: 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
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 BurgessAuthored: 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
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 LimAuthored: 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
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 VillardAuthored: 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
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 BurgessAuthored: 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
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 BurgessAuthored: 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
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 BurgessAuthored: 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
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 BurgessAuthored: 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.
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 PiriAuthored: 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
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 VillardAuthored: 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
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 ZhurakouskyAuthored: 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
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 FrazeeAuthored: 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
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 FrazeeAuthored: 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
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 ZhurakouskyAuthored: 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.
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 GilmanAuthored: 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
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 MirandaAuthored: 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
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 VillardAuthored: 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
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 VillardAuthored: 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
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 ZhurakouskyAuthored: 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
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 PayneAuthored: 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.
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 LoPrestoAuthored: 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
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 StorckAuthored: 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
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 MirandaAuthored: 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.
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 ZhurakouskyAuthored: 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
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 AslanAuthored: 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:
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. DavisAuthored: 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.
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 PayneAuthored: 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
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
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
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 BurgessAuthored: 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
NIFI-2423: Make use of the SSLContextService to provide SSL information Signed-off-by: Oleg ZhurakouskyProject: 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/
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 ZhurakouskyAuthored: 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/
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 ZhurakouskyAuthored: 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
Repository: nifi Updated Branches: refs/heads/master 8d380dcda -> 7ffa30d21 NIFI-2423: Make use of the SSLContextService to provide SSL information Signed-off-by: Oleg ZhurakouskyProject: 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
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 ZhurakouskyAuthored: 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)
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 ZhurakouskyAuthored: 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
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 BendeAuthored: 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
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: jpercivallAuthored: 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.
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: jpercivallAuthored: 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
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 VillardAuthored: 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
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 VillardAuthored: 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
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 VillardAuthored: 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
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
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 VillardAuthored: 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
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
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: JoshiAuthored: 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) {