You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2015/02/04 19:56:55 UTC

[01/36] incubator-nifi git commit: NIFI-305: Refactoring superclass BinFiles from MergeContent

Repository: incubator-nifi
Updated Branches:
  refs/heads/NIFI-250 13fb1a758 -> 6b91546d9


NIFI-305: Refactoring superclass BinFiles from MergeContent


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

Branch: refs/heads/NIFI-250
Commit: 98afcce0dc3672fe04e2130ac72b9a201e17ba1a
Parents: 6b560b9
Author: gresockj <jg...@gmail.com>
Authored: Tue Jan 27 17:32:38 2015 -0500
Committer: gresockj <jg...@gmail.com>
Committed: Tue Jan 27 17:32:38 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/BinFiles.java      | 388 ++++++++++++++++++
 .../nifi/processors/standard/MergeContent.java  | 400 ++++---------------
 2 files changed, 473 insertions(+), 315 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/98afcce0/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
new file mode 100644
index 0000000..7846c7d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
@@ -0,0 +1,388 @@
+/*
+ * 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.standard;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ProcessorLog;
+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;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.standard.util.Bin;
+import org.apache.nifi.processors.standard.util.BinManager;
+import org.apache.nifi.processors.standard.util.FlowFileSessionWrapper;
+
+/**
+ * Base class for MergeContent.
+ *
+ */
+public abstract class BinFiles extends AbstractSessionFactoryProcessor {
+
+    public static final PropertyDescriptor MIN_SIZE = new PropertyDescriptor.Builder()
+            .name("Minimum Group Size")
+            .description("The minimum size of for the bundle")
+            .required(true)
+            .defaultValue("0 B")
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MAX_SIZE = new PropertyDescriptor.Builder()
+            .name("Maximum Group Size")
+            .description("The maximum size for the bundle. If not specified, there is no maximum.")
+            .required(false)
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MIN_ENTRIES = new PropertyDescriptor.Builder()
+            .name("Minimum Number of Entries")
+            .description("The minimum number of files to include in a bundle")
+            .required(true)
+            .defaultValue("1")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MAX_ENTRIES = new PropertyDescriptor.Builder()
+            .name("Maximum Number of Entries")
+            .description("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MAX_BIN_COUNT = new PropertyDescriptor.Builder()
+            .name("Maximum number of Bins")
+            .description("Specifies the maximum number of bins that can be held in memory at any one time")
+            .defaultValue("100")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MAX_BIN_AGE = new PropertyDescriptor.Builder()
+            .name("Max Bin Age")
+            .description("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit> where <duration> is a positive integer and time unit is one of seconds, minutes, hours")
+            .required(false)
+            .addValidator(StandardValidators.createTimePeriodValidator(1, TimeUnit.SECONDS, Integer.MAX_VALUE, TimeUnit.SECONDS))
+            .build();
+
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder().name("original").description("The FlowFiles that were used to create the bundle").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("If the bundle cannot be created, all FlowFiles that would have been used to created the bundle will be transferred to failure").build();
+
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> descriptors;
+    private final BinManager binManager = new BinManager();
+
+    private final Queue<Bin> readyBins = new LinkedBlockingQueue<>();
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+
+    	final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        Set<Relationship> additionalRelationships = defineAdditionalRelationships();
+        if (additionalRelationships != null) {
+        	relationships.addAll(additionalRelationships);
+        }
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(MIN_ENTRIES);
+        descriptors.add(MAX_ENTRIES);
+        descriptors.add(MIN_SIZE);
+        descriptors.add(MAX_SIZE);
+        descriptors.add(MAX_BIN_AGE);
+        descriptors.add(MAX_BIN_COUNT);
+        List<PropertyDescriptor> additionalPropertyDescriptors = this.defineAdditionalPropertyDescriptors();
+        if (additionalPropertyDescriptors != null) {
+        	descriptors.addAll(additionalPropertyDescriptors);
+        }
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+    }
+
+    @OnStopped
+    public void resetState() {
+        binManager.purge();
+
+        Bin bin;
+        while ((bin = readyBins.poll()) != null) {
+            for (final FlowFileSessionWrapper wrapper : bin.getContents()) {
+                wrapper.getSession().rollback();
+            }
+        }
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    /**
+     * Allows any additional relationships to be defined.
+     * @return Relationships to be added in the init() method
+     */ 
+    protected abstract Set<Relationship> defineAdditionalRelationships();
+    
+    /**
+     * Allows any additional property descriptors to be defined.
+     * @return Properties to be added in the init() method
+     */
+    protected abstract List<PropertyDescriptor> defineAdditionalPropertyDescriptors();
+
+	/**
+	 * Allows general pre-processing of a flow file before it is offered to a
+	 * bin. This is called before getGroupId().
+	 * 
+	 * @param context
+	 * @param session
+	 * @param flowFile
+	 * @return The flow file, possibly altered
+	 */
+    protected abstract FlowFile preprocessFlowFile(final ProcessContext context, final ProcessSession session, final FlowFile flowFile);
+    
+    /**
+     * Returns a group ID representing a bin.  This allows flow files to be
+     * binned into like groups.
+     * @param context 
+     * @param flowFile
+     * @return The appropriate group ID
+     */
+    protected abstract String getGroupId(final ProcessContext context, final FlowFile flowFile);
+
+    /**
+     * Performs any additional setup of the bin manager.  Called during the
+     * OnScheduled phase.
+     * @param binManager The bin manager
+     * @param context
+     */
+    protected abstract void setUpBinManager(BinManager binManager, ProcessContext context);
+    
+    /**
+	 * Processes a single bin. Implementing class is responsible for committing
+	 * each session
+	 * 
+	 * @param unmodifiableBin
+	 *            A reference to a single bin of flow file/session wrappers
+	 * @param binContents
+	 *            A copy of the contents of the bin
+	 * @param context
+	 *            The context
+	 * @param session
+	 *            The session that created the bin
+	 * @param logger
+	 *            The logger
+	 * @return Return true if the input bin was already committed. E.g., in case of a
+	 * failure, the implementation may choose to transfer all binned files
+	 * to Failure and commit their sessions.  If false, the 
+	 * processBins() method will transfer the files to Original and commit
+	 * the sessions
+	 * @throws Exception
+	 *             This will be handled appropriately, and all flow files in the
+	 *             bin will be transferred to failure and the session rolled
+	 *             back
+	 */
+	protected abstract boolean processBin(Bin unmodifiableBin,
+			List<FlowFileSessionWrapper> binContents, ProcessContext context,
+			ProcessSession session, ProcessorLog logger) throws Exception;
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+        int binsAdded = binFlowFiles(context, sessionFactory);
+        getLogger().debug("Binned {} FlowFiles", new Object[] {binsAdded});
+        
+        if (!isScheduled()) {
+            return;
+        }
+
+        binsAdded += migrateBins(context);
+
+        final int binsProcessed = processBins(context, sessionFactory);
+        if (binsProcessed == 0 && binsAdded == 0) {
+            context.yield();
+        }
+    }
+
+    private int migrateBins(final ProcessContext context) {
+        int added = 0;
+        for (final Bin bin : binManager.removeReadyBins(true)) {
+            this.readyBins.add(bin);
+            added++;
+        }
+
+        // if we have created all of the bins that are allowed, go ahead and remove the oldest one. If we don't do
+        // this, then we will simply wait for it to expire because we can't get any more FlowFiles into the
+        // bins. So we may as well expire it now.
+        if (added == 0 && binManager.getBinCount() >= context.getProperty(MAX_BIN_COUNT).asInteger()) {
+            final Bin bin = binManager.removeOldestBin();
+            if (bin != null) {
+                added++;
+                this.readyBins.add(bin);
+            }
+        }
+
+        return added;
+    }
+
+    private int processBins(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        final Bin bin = readyBins.poll();
+        if (bin == null) {
+            return 0;
+        }
+
+        final List<Bin> bins = new ArrayList<>();
+        bins.add(bin);
+
+        final ProcessorLog logger = getLogger();
+        final ProcessSession session = sessionFactory.createSession();
+
+        final List<FlowFileSessionWrapper> binCopy = new ArrayList<>(bin.getContents());
+
+    	boolean binAlreadyCommitted = false;
+        try {
+        	binAlreadyCommitted = this.processBin(bin, binCopy, context, session, logger);
+        } catch (final Exception e) {
+            logger.error("Failed to process bundle of {} files due to {}", new Object[]{binCopy.size(), e});
+
+            for (final FlowFileSessionWrapper wrapper : binCopy) {
+                wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
+                wrapper.getSession().commit();
+            }
+            session.rollback();
+            return 1;
+        }
+
+        // we first commit the bundle's session before the originals' sessions because if we are restarted or crash
+        // between commits, we favor data redundancy over data loss. Since we have no Distributed Transaction capability
+        // across multiple sessions, we cannot guarantee atomicity across the sessions
+        session.commit();
+        // If this bin's session has been committed, move on.
+        if ( !binAlreadyCommitted ) {
+            for (final FlowFileSessionWrapper wrapper : bin.getContents()) {
+                wrapper.getSession().transfer(wrapper.getFlowFile(), REL_ORIGINAL);
+                wrapper.getSession().commit();
+            }
+        }
+
+        return 1;
+    }
+    
+	private int binFlowFiles(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        int binsAdded = 0;
+        while (binManager.getBinCount() < context.getProperty(MAX_BIN_COUNT).asInteger().intValue()) {
+            if (!isScheduled()) {
+                return binsAdded;
+            }
+
+            final ProcessSession session = sessionFactory.createSession();
+            FlowFile flowFile = session.get();
+            if (flowFile == null) {
+                return binsAdded;
+            }
+
+            flowFile = this.preprocessFlowFile(context, session, flowFile);
+
+            String groupId = this.getGroupId(context, flowFile);
+
+            final boolean binned = binManager.offer(groupId, flowFile, session);
+
+            // could not be added to a bin -- probably too large by itself, so create a separate bin for just this guy.
+            if (!binned) {
+                Bin bin = new Bin(0, Long.MAX_VALUE, 0, Integer.MAX_VALUE, null);
+                bin.offer(flowFile, session);
+                this.readyBins.add(bin);
+            }
+
+            binsAdded++;
+        }
+
+        return binsAdded;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws IOException {
+        binManager.setMinimumSize(context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue());
+
+        if (context.getProperty(MAX_BIN_AGE).isSet() ) {
+            binManager.setMaxBinAge(context.getProperty(MAX_BIN_AGE).asTimePeriod(TimeUnit.SECONDS).intValue());
+        } else {
+            binManager.setMaxBinAge(Integer.MAX_VALUE);
+        }
+        
+        if ( context.getProperty(MAX_SIZE).isSet() ) {
+            binManager.setMaximumSize(context.getProperty(MAX_SIZE).asDataSize(DataUnit.B).longValue());
+        } else {
+            binManager.setMaximumSize(Long.MAX_VALUE);
+        }
+        
+        binManager.setMinimumEntries(context.getProperty(MIN_ENTRIES).asInteger());
+
+        if ( context.getProperty(MAX_ENTRIES).isSet() ) {
+            binManager.setMaximumEntries(context.getProperty(MAX_ENTRIES).asInteger().intValue());
+        } else {
+            binManager.setMaximumEntries(Integer.MAX_VALUE);
+        }
+
+        this.setUpBinManager(binManager, context);
+    }
+    
+	@Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        final List<ValidationResult> problems = new ArrayList<>(super.customValidate(context));
+
+        final long minBytes = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
+        final Double maxBytes = context.getProperty(MAX_SIZE).asDataSize(DataUnit.B);
+
+        if (maxBytes != null && maxBytes.longValue() < minBytes) {
+            problems.add(new ValidationResult.Builder().subject(MIN_SIZE.getName()).input(
+                    context.getProperty(MIN_SIZE).getValue()).valid(false).explanation("Min Size must be less than or equal to Max Size").build());
+        }
+
+        final Long min = context.getProperty(MIN_ENTRIES).asLong();
+        final Long max = context.getProperty(MAX_ENTRIES).asLong();
+
+        if (min != null && max != null) {
+            if (min > max) {
+                problems.add(new ValidationResult.Builder().subject(MIN_ENTRIES.getName()).input(context.getProperty(MIN_ENTRIES).getValue()).valid(false).explanation("Min Entries must be less than or equal to Max Entries").build());
+            }
+        }
+
+        return problems;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/98afcce0/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index f2e4a8d..73cb5a6 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -23,7 +23,6 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -31,59 +30,47 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
 import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipOutputStream;
 
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.PropertyValue;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.NonCloseableOutputStream;
-import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.logging.ProcessorLog;
-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;
-import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
-import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.standard.util.Bin;
 import org.apache.nifi.processors.standard.util.BinManager;
 import org.apache.nifi.processors.standard.util.FlowFileSessionWrapper;
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.NonCloseableOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.FlowFilePackager;
 import org.apache.nifi.util.FlowFilePackagerV1;
 import org.apache.nifi.util.FlowFilePackagerV2;
 import org.apache.nifi.util.FlowFilePackagerV3;
 import org.apache.nifi.util.ObjectHolder;
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
 
 @SideEffectFree
 @TriggerWhenEmpty
 @Tags({"merge", "content", "correlation", "tar", "zip", "stream", "concatenation", "archive", "flowfile-stream", "flowfile-stream-v3"})
 @CapabilityDescription("Merges a Group of FlowFiles together based on a user-defined strategy and packages them into a single FlowFile. It is recommended that the Processor be configured with only a single incoming connection, as Group of FlowFiles will not be created from FlowFiles in different connections. This processor updates the mime.type attribute as appropriate.")
-public class MergeContent extends AbstractSessionFactoryProcessor {
+public class MergeContent extends BinFiles {
 
     // preferred attributes
     public static final String FRAGMENT_ID_ATTRIBUTE = "fragment.identifier";
@@ -207,160 +194,82 @@ public class MergeContent extends AbstractSessionFactoryProcessor {
             .defaultValue("false")
             .build();
 
-    public static final PropertyDescriptor MIN_SIZE = new PropertyDescriptor.Builder()
-            .name("Minimum Group Size")
-            .description("The minimum size of for the bundle")
-            .required(true)
-            .defaultValue("0 B")
-            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .build();
-    public static final PropertyDescriptor MAX_SIZE = new PropertyDescriptor.Builder()
-            .name("Maximum Group Size")
-            .description("The maximum size for the bundle. If not specified, there is no maximum.")
-            .required(false)
-            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor MIN_ENTRIES = new PropertyDescriptor.Builder()
-            .name("Minimum Number of Entries")
-            .description("The minimum number of files to include in a bundle")
-            .required(true)
-            .defaultValue("1")
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .build();
-    public static final PropertyDescriptor MAX_ENTRIES = new PropertyDescriptor.Builder()
-            .name("Maximum Number of Entries")
-            .description("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
-            .required(false)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor MAX_BIN_COUNT = new PropertyDescriptor.Builder()
-            .name("Maximum number of Bins")
-            .description("Specifies the maximum number of bins that can be held in memory at any one time")
-            .defaultValue("100")
-            .required(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor MAX_BIN_AGE = new PropertyDescriptor.Builder()
-            .name("Max Bin Age")
-            .description("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit> where <duration> is a positive integer and time unit is one of seconds, minutes, hours")
-            .required(false)
-            .addValidator(StandardValidators.createTimePeriodValidator(1, TimeUnit.SECONDS, Integer.MAX_VALUE, TimeUnit.SECONDS))
-            .build();
-
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder().name("original").description("The FlowFiles that were used to create the bundle").build();
     public static final Relationship REL_MERGED = new Relationship.Builder().name("merged").description("The FlowFile containing the merged content").build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("If the bundle cannot be created, all FlowFiles that would have been used to created the bundle will be transferred to failure").build();
 
     public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+");
 
-    private Set<Relationship> relationships;
-    private List<PropertyDescriptor> descriptors;
-    private final BinManager binManager = new BinManager();
-
-    private final Queue<Bin> readyBins = new LinkedBlockingQueue<>();
-
-    @Override
-    protected void init(final ProcessorInitializationContext context) {
+	@Override
+	protected Set<Relationship> defineAdditionalRelationships() {
         final Set<Relationship> relationships = new HashSet<>();
-        relationships.add(REL_ORIGINAL);
         relationships.add(REL_MERGED);
-        relationships.add(REL_FAILURE);
-        this.relationships = Collections.unmodifiableSet(relationships);
+        
+        return relationships;
+    }
 
-        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+	@Override
+	protected List<PropertyDescriptor> defineAdditionalPropertyDescriptors() {
+		final List<PropertyDescriptor> descriptors = new ArrayList<>();
         descriptors.add(MERGE_STRATEGY);
         descriptors.add(MERGE_FORMAT);
         descriptors.add(ATTRIBUTE_STRATEGY);
         descriptors.add(CORRELATION_ATTRIBUTE_NAME);
-        descriptors.add(MIN_ENTRIES);
-        descriptors.add(MAX_ENTRIES);
-        descriptors.add(MIN_SIZE);
-        descriptors.add(MAX_SIZE);
-        descriptors.add(MAX_BIN_AGE);
-        descriptors.add(MAX_BIN_COUNT);
         descriptors.add(HEADER);
         descriptors.add(FOOTER);
         descriptors.add(DEMARCATOR);
         descriptors.add(COMPRESSION_LEVEL);
         descriptors.add(KEEP_PATH);
-
-        this.descriptors = Collections.unmodifiableList(descriptors);
-    }
-
-    @OnStopped
-    public void resetState() {
-        binManager.purge();
-
-        Bin bin;
-        while ((bin = readyBins.poll()) != null) {
-            for (final FlowFileSessionWrapper wrapper : bin.getContents()) {
-                wrapper.getSession().rollback();
-            }
-        }
-    }
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        
         return descriptors;
-    }
+	}
 
     private byte[] readContent(final String filename) throws IOException {
         return Files.readAllBytes(Paths.get(filename));
     }
 
-    
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        int binsAdded = binFlowFiles(context, sessionFactory);
-        getLogger().debug("Binned {} FlowFiles", new Object[] {binsAdded});
-        
-        if (!isScheduled()) {
-            return;
+
+	@Override
+	protected FlowFile preprocessFlowFile(ProcessContext context,
+			ProcessSession session, FlowFile flowFile) {
+		
+        // handle backward compatibility with old segment attributes
+        if (flowFile.getAttribute(FRAGMENT_COUNT_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_COUNT_ATTRIBUTE) != null) {
+            flowFile = session.putAttribute(flowFile, FRAGMENT_COUNT_ATTRIBUTE, flowFile.getAttribute(SEGMENT_COUNT_ATTRIBUTE));
+        }
+        if (flowFile.getAttribute(FRAGMENT_INDEX_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_INDEX_ATTRIBUTE) != null) {
+            flowFile = session.putAttribute(flowFile, FRAGMENT_INDEX_ATTRIBUTE, flowFile.getAttribute(SEGMENT_INDEX_ATTRIBUTE));
         }
+        if (flowFile.getAttribute(FRAGMENT_ID_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_ID_ATTRIBUTE) != null) {
+            flowFile = session.putAttribute(flowFile, FRAGMENT_ID_ATTRIBUTE, flowFile.getAttribute(SEGMENT_ID_ATTRIBUTE));
+        }
+        
+        return flowFile;
+	}
 
-        binsAdded += migrateBins(context);
+	@Override
+	protected String getGroupId(ProcessContext context, FlowFile flowFile) {
 
-        final int binsProcessed = processBins(context, sessionFactory);
-        if (binsProcessed == 0 && binsAdded == 0) {
-            context.yield();
-        }
-    }
-    
-
-    private int migrateBins(final ProcessContext context) {
-        int added = 0;
-        for (final Bin bin : binManager.removeReadyBins(true)) {
-            this.readyBins.add(bin);
-            added++;
-        }
-
-        // if we have created all of the bins that are allowed, go ahead and remove the oldest one. If we don't do
-        // this, then we will simply wait for it to expire because we can't get any more FlowFiles into the
-        // bins. So we may as well expire it now.
-        if (added == 0 && binManager.getBinCount() >= context.getProperty(MAX_BIN_COUNT).asInteger()) {
-            final Bin bin = binManager.removeOldestBin();
-            if (bin != null) {
-                added++;
-                this.readyBins.add(bin);
-            }
+        final String correlationAttributeName = context.getProperty(CORRELATION_ATTRIBUTE_NAME).getValue();
+        String groupId = (correlationAttributeName == null) ? null : flowFile.getAttribute(correlationAttributeName);
+
+        // when MERGE_STRATEGY is Defragment and correlationAttributeName is null then bin by fragment.identifier
+        if (groupId == null && MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
+            groupId = flowFile.getAttribute(FRAGMENT_ID_ATTRIBUTE);
         }
 
-        return added;
-    }
+        return groupId;
+	}
 
-    private int processBins(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
-        final Bin bin = readyBins.poll();
-        if (bin == null) {
-            return 0;
+	@Override
+	protected void setUpBinManager(BinManager binManager, ProcessContext context) {
+		if (MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
+            binManager.setFileCountAttribute(FRAGMENT_COUNT_ATTRIBUTE);
         }
+	}
+
+	@Override
+	protected boolean processBin(Bin unmodifiableBin,
+			List<FlowFileSessionWrapper> binCopy, ProcessContext context,
+			ProcessSession session, ProcessorLog logger) throws Exception {
 
         final String mergeFormat = context.getProperty(MERGE_FORMAT).getValue();
         MergeBin merger;
@@ -398,130 +307,45 @@ public class MergeContent extends AbstractSessionFactoryProcessor {
                 break;
         }
 
-        final List<Bin> bins = new ArrayList<>();
-        bins.add(bin);
-
-        final ProcessorLog logger = getLogger();
-        final ProcessSession session = sessionFactory.createSession();
-
-        final Set<Bin> committedBins = new HashSet<>();
-        
-        for (final Bin unmodifiableBin : bins) {
-            final List<FlowFileSessionWrapper> binCopy = new ArrayList<>(unmodifiableBin.getContents());
-
-            if (MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
-                final String error = getDefragmentValidationError(binCopy);
-                if (error != null) {
-                    final String binDescription = binCopy.size() <= 10 ? binCopy.toString() : binCopy.size() + " FlowFiles";
-                    logger.error(error + "; routing {} to failure", new Object[]{binDescription});
-                    for ( final FlowFileSessionWrapper wrapper : binCopy ) {
-                        wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
-                        wrapper.getSession().commit();
-                        committedBins.add(unmodifiableBin);
-                    }
-                    
-                    continue;
-                }
-
-                Collections.sort(binCopy, new FragmentComparator());
-            }
-
-            FlowFile bundle = null;
-            try {
-                bundle = merger.merge(context, session, binCopy);
-
-                // keep the filename, as it is added to the bundle.
-                final String filename = bundle.getAttribute(CoreAttributes.FILENAME.key());
-
-                // merge all of the attributes
-                final Map<String, String> bundleAttributes = attributeStrategy.getMergedAttributes(binCopy);
-                bundleAttributes.put(CoreAttributes.MIME_TYPE.key(), merger.getMergedContentType());
-                // restore the filename of the bundle
-                bundleAttributes.put(CoreAttributes.FILENAME.key(), filename);
-                bundleAttributes.put(MERGE_COUNT_ATTRIBUTE, Integer.toString(binCopy.size()));
-                bundleAttributes.put(MERGE_BIN_AGE_ATTRIBUTE, Long.toString(bin.getBinAge()));
-
-                bundle = session.putAllAttributes(bundle, bundleAttributes);
-
-                final String inputDescription = (binCopy.size() < 10) ? binCopy.toString() : binCopy.size() + " FlowFiles";
-                logger.info("Merged {} into {}", new Object[]{inputDescription, bundle});
-            } catch (final Exception e) {
-                logger.error("Failed to process bundle of {} files due to {}", new Object[]{binCopy.size(), e});
 
-                for (final FlowFileSessionWrapper wrapper : binCopy) {
+        if (MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
+            final String error = getDefragmentValidationError(binCopy);
+            
+            // Fail the flow files and commit them
+            if (error != null) {
+                final String binDescription = binCopy.size() <= 10 ? binCopy.toString() : binCopy.size() + " FlowFiles";
+                logger.error(error + "; routing {} to failure", new Object[]{binDescription});
+                for ( final FlowFileSessionWrapper wrapper : binCopy ) {
                     wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
                     wrapper.getSession().commit();
                 }
-                session.rollback();
-                return 1;
-            }
-            session.transfer(bundle, REL_MERGED);
-        }
-
-        // we first commit the bundle's session before the originals' sessions because if we are restarted or crash
-        // between commits, we favor data redundancy over data loss. Since we have no Distributed Transaction capability
-        // across multiple sessions, we cannot guarantee atomicity across the sessions
-        session.commit();
-        for (final Bin unmodifiableBin : bins) {
-            // If this bin's session has been committed, move on.
-            if ( committedBins.contains(unmodifiableBin) ) {
-                continue;
-            }
-            
-            for (final FlowFileSessionWrapper wrapper : unmodifiableBin.getContents()) {
-                wrapper.getSession().transfer(wrapper.getFlowFile(), REL_ORIGINAL);
-                wrapper.getSession().commit();
+                
+                return true;
             }
+            Collections.sort(binCopy, new FragmentComparator());
         }
 
-        return 1;
-    }
-
-    private int binFlowFiles(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
-        int binsAdded = 0;
-        while (binManager.getBinCount() < context.getProperty(MAX_BIN_COUNT).asInteger().intValue()) {
-            if (!isScheduled()) {
-                return binsAdded;
-            }
-
-            final ProcessSession session = sessionFactory.createSession();
-            FlowFile flowFile = session.get();
-            if (flowFile == null) {
-                return binsAdded;
-            }
+        FlowFile bundle = merger.merge(context, session, binCopy);
 
-            // handle backward compatibility with old segment attributes
-            if (flowFile.getAttribute(FRAGMENT_COUNT_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_COUNT_ATTRIBUTE) != null) {
-                flowFile = session.putAttribute(flowFile, FRAGMENT_COUNT_ATTRIBUTE, flowFile.getAttribute(SEGMENT_COUNT_ATTRIBUTE));
-            }
-            if (flowFile.getAttribute(FRAGMENT_INDEX_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_INDEX_ATTRIBUTE) != null) {
-                flowFile = session.putAttribute(flowFile, FRAGMENT_INDEX_ATTRIBUTE, flowFile.getAttribute(SEGMENT_INDEX_ATTRIBUTE));
-            }
-            if (flowFile.getAttribute(FRAGMENT_ID_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_ID_ATTRIBUTE) != null) {
-                flowFile = session.putAttribute(flowFile, FRAGMENT_ID_ATTRIBUTE, flowFile.getAttribute(SEGMENT_ID_ATTRIBUTE));
-            }
+        // keep the filename, as it is added to the bundle.
+        final String filename = bundle.getAttribute(CoreAttributes.FILENAME.key());
 
-            final String correlationAttributeName = context.getProperty(CORRELATION_ATTRIBUTE_NAME).getValue();
-            String groupId = (correlationAttributeName == null) ? null : flowFile.getAttribute(correlationAttributeName);
+        // merge all of the attributes
+        final Map<String, String> bundleAttributes = attributeStrategy.getMergedAttributes(binCopy);
+        bundleAttributes.put(CoreAttributes.MIME_TYPE.key(), merger.getMergedContentType());
+        // restore the filename of the bundle
+        bundleAttributes.put(CoreAttributes.FILENAME.key(), filename);
+        bundleAttributes.put(MERGE_COUNT_ATTRIBUTE, Integer.toString(binCopy.size()));
+        bundleAttributes.put(MERGE_BIN_AGE_ATTRIBUTE, Long.toString(unmodifiableBin.getBinAge()));
 
-            // when MERGE_STRATEGY is Defragment and correlationAttributeName is null then bin by fragment.identifier
-            if (groupId == null && MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
-                groupId = flowFile.getAttribute(FRAGMENT_ID_ATTRIBUTE);
-            }
+        bundle = session.putAllAttributes(bundle, bundleAttributes);
 
-            final boolean binned = binManager.offer(groupId, flowFile, session);
+        final String inputDescription = (binCopy.size() < 10) ? binCopy.toString() : binCopy.size() + " FlowFiles";
+        logger.info("Merged {} into {}", new Object[]{inputDescription, bundle});
+        session.transfer(bundle, REL_MERGED);
 
-            // could not be added to a bin -- probably too large by itself, so create a separate bin for just this guy.
-            if (!binned) {
-                Bin bin = new Bin(0, Long.MAX_VALUE, 0, Integer.MAX_VALUE, null);
-                bin.offer(flowFile, session);
-                this.readyBins.add(bin);
-            }
-
-            binsAdded++;
-        }
-
-        return binsAdded;
+        // We haven't committed anything, parent will take care of it
+        return false;
     }
 
     private String getDefragmentValidationError(final List<FlowFileSessionWrapper> bin) {
@@ -578,60 +402,6 @@ public class MergeContent extends AbstractSessionFactoryProcessor {
         return NUMBER_PATTERN.matcher(value).matches();
     }
 
-    @OnScheduled
-    public void onScheduled(final ProcessContext context) throws IOException {
-        binManager.setMinimumSize(context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue());
-
-        if (context.getProperty(MAX_BIN_AGE).isSet() ) {
-            binManager.setMaxBinAge(context.getProperty(MAX_BIN_AGE).asTimePeriod(TimeUnit.SECONDS).intValue());
-        } else {
-            binManager.setMaxBinAge(Integer.MAX_VALUE);
-        }
-        
-        if ( context.getProperty(MAX_SIZE).isSet() ) {
-            binManager.setMaximumSize(context.getProperty(MAX_SIZE).asDataSize(DataUnit.B).longValue());
-        } else {
-            binManager.setMaximumSize(Long.MAX_VALUE);
-        }
-        
-        if (MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
-            binManager.setFileCountAttribute(FRAGMENT_COUNT_ATTRIBUTE);
-        } else {
-            binManager.setMinimumEntries(context.getProperty(MIN_ENTRIES).asInteger());
-
-            if ( context.getProperty(MAX_ENTRIES).isSet() ) {
-                binManager.setMaximumEntries(context.getProperty(MAX_ENTRIES).asInteger().intValue());
-            } else {
-                binManager.setMaximumEntries(Integer.MAX_VALUE);
-            }
-        }
-
-    }
-
-    @Override
-    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
-        final List<ValidationResult> problems = new ArrayList<>(super.customValidate(context));
-
-        final long minBytes = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
-        final Double maxBytes = context.getProperty(MAX_SIZE).asDataSize(DataUnit.B);
-
-        if (maxBytes != null && maxBytes.longValue() < minBytes) {
-            problems.add(new ValidationResult.Builder().subject(MIN_SIZE.getName()).input(
-                    context.getProperty(MIN_SIZE).getValue()).valid(false).explanation("Min Size must be less than or equal to Max Size").build());
-        }
-
-        final Long min = context.getProperty(MIN_ENTRIES).asLong();
-        final Long max = context.getProperty(MAX_ENTRIES).asLong();
-
-        if (min != null && max != null) {
-            if (min > max) {
-                problems.add(new ValidationResult.Builder().subject(MIN_ENTRIES.getName()).input(context.getProperty(MIN_ENTRIES).getValue()).valid(false).explanation("Min Entries must be less than or equal to Max Entries").build());
-            }
-        }
-
-        return problems;
-    }
-
     private class BinaryConcatenationMerge implements MergeBin {
 
         private String mimeType = "application/octet-stream";


[08/36] incubator-nifi git commit: Merge branch 'NIFI-305' of https://github.com/gresockj/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'NIFI-305' of https://github.com/gresockj/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: c01dff5922239a28d02d407fb86f27f110aceeab
Parents: bafa945 615794e
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 2 13:23:36 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 2 13:23:36 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/BinFiles.java      | 405 +++++++++++++++++++
 .../nifi/processors/standard/MergeContent.java  | 400 ++++--------------
 2 files changed, 490 insertions(+), 315 deletions(-)
----------------------------------------------------------------------



[30/36] incubator-nifi git commit: NIFI-162: Updating gitignore

Posted by mc...@apache.org.
NIFI-162: Updating gitignore


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

Branch: refs/heads/NIFI-250
Commit: b0b14ed10b79ab3d8d6caa0aef2d337b42cfefdd
Parents: e45dea1
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 4 07:20:49 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 4 07:20:49 2015 -0500

----------------------------------------------------------------------
 .gitignore | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b0b14ed1/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index e1c837c..5dacd2a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -13,3 +13,10 @@ nb-configuration.xml
 *.iml
 *.iws
 *~
+
+# nifi site
+/nifi-site/nbproject/
+/nifi-site/bower_components/
+/nifi-site/dist/
+/nifi-site/node_modules/
+/nifi-site/.sass-cache


[23/36] incubator-nifi git commit: NIFI-310: Fixed NPE

Posted by mc...@apache.org.
NIFI-310: Fixed NPE


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

Branch: refs/heads/NIFI-250
Commit: 102e3cb093cd7380ffaf5ebff80a8fe64485c22b
Parents: be16371
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 3 10:37:28 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 3 10:37:28 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/PostHTTP.java      | 30 ++++++++++++--------
 1 file changed, 18 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/102e3cb0/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
index fd486b0..f8a33bc 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
@@ -68,6 +68,7 @@ import org.apache.http.conn.HttpClientConnectionManager;
 import org.apache.http.conn.ManagedHttpClientConnection;
 import org.apache.http.conn.socket.ConnectionSocketFactory;
 import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLContextBuilder;
 import org.apache.http.conn.ssl.SSLContexts;
 import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
 import org.apache.http.entity.ContentProducer;
@@ -352,21 +353,26 @@ public class PostHTTP extends AbstractProcessor {
     private SSLContext createSSLContext(final SSLContextService service) throws KeyStoreException, IOException, NoSuchAlgorithmException, 
         CertificateException, KeyManagementException, UnrecoverableKeyException 
     {
-        final KeyStore truststore  = KeyStore.getInstance(service.getTrustStoreType());
-        try (final InputStream in = new FileInputStream(new File(service.getTrustStoreFile()))) {
-            truststore.load(in, service.getTrustStorePassword().toCharArray());
+        SSLContextBuilder builder = SSLContexts.custom();
+        final String trustFilename = service.getTrustStoreFile();
+        if ( trustFilename != null ) {
+            final KeyStore truststore  = KeyStore.getInstance(service.getTrustStoreType());
+            try (final InputStream in = new FileInputStream(new File(service.getTrustStoreFile()))) {
+                truststore.load(in, service.getTrustStorePassword().toCharArray());
+            }
+            builder = builder.loadTrustMaterial(truststore, new TrustSelfSignedStrategy());
         }
-        
-        final KeyStore keystore  = KeyStore.getInstance(service.getKeyStoreType());
-        try (final InputStream in = new FileInputStream(new File(service.getKeyStoreFile()))) {
-            keystore.load(in, service.getKeyStorePassword().toCharArray());
+
+        final String keyFilename = service.getKeyStoreFile();
+        if ( keyFilename != null ) {
+            final KeyStore keystore  = KeyStore.getInstance(service.getKeyStoreType());
+            try (final InputStream in = new FileInputStream(new File(service.getKeyStoreFile()))) {
+                keystore.load(in, service.getKeyStorePassword().toCharArray());
+            }
+            builder = builder.loadKeyMaterial(keystore, service.getKeyStorePassword().toCharArray());
         }
         
-        SSLContext sslContext = SSLContexts.custom()
-                .loadTrustMaterial(truststore, new TrustSelfSignedStrategy())
-                .loadKeyMaterial(keystore, service.getKeyStorePassword().toCharArray())
-                .build();
-        
+        SSLContext sslContext = builder.build();
         return sslContext;
     }
 


[05/36] incubator-nifi git commit: NIFI-307: - Updating how action buttons are invoked in tables throughout the application.

Posted by mc...@apache.org.
NIFI-307:
- Updating how action buttons are invoked in tables throughout the application.

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

Branch: refs/heads/NIFI-250
Commit: 9cd9d126e34a3039fcf42215c4ad6e67551c8f25
Parents: 037f36d
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 12:44:51 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 12:44:51 2015 -0500

----------------------------------------------------------------------
 .../webapp/js/nf/cluster/nf-cluster-table.js    | 319 +++++++-------
 .../webapp/js/nf/counters/nf-counters-table.js  |  63 +--
 .../webapp/js/nf/history/nf-history-table.js    | 147 ++++---
 .../js/nf/provenance/nf-provenance-table.js     | 103 ++---
 .../js/nf/templates/nf-templates-table.js       |  73 ++-
 .../main/webapp/js/nf/users/nf-users-table.js   | 440 +++++++++----------
 6 files changed, 562 insertions(+), 583 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9cd9d126/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js
index 13b0e5f..2968919 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js
@@ -110,6 +110,22 @@ nf.ClusterTable = (function () {
     };
 
     /**
+     * Prompts to verify node connection.
+     * 
+     * @argument {object} node     The node
+     */
+    var promptForConnect = function (node) {
+        // prompt to connect
+        nf.Dialog.showYesNoDialog({
+            dialogContent: 'Connect \'' + formatNodeAddress(node) + '\' to this cluster?',
+            overlayBackground: false,
+            yesHandler: function () {
+                connect(node.nodeId);
+            }
+        });
+    };
+
+    /**
      * Connects the node in the specified row.
      * 
      * @argument {string} nodeId     The node id
@@ -133,6 +149,22 @@ nf.ClusterTable = (function () {
     };
 
     /**
+     * Prompts to verify node disconnection.
+     * 
+     * @argument {object} node     The node
+     */
+    var promptForDisconnect = function (node) {
+        // prompt for disconnect
+        nf.Dialog.showYesNoDialog({
+            dialogContent: 'Disconnect \'' + formatNodeAddress(node) + '\' from the cluster?',
+            overlayBackground: false,
+            yesHandler: function () {
+                disconnect(node.nodeId);
+            }
+        });
+    };
+
+    /**
      * Disconnects the node in the specified row.
      * 
      * @argument {string} nodeId     The node id
@@ -156,6 +188,22 @@ nf.ClusterTable = (function () {
     };
 
     /**
+     * Prompts to verify node disconnection.
+     * 
+     * @argument {object} node     The node
+     */
+    var promptForRemoval = function (node) {
+        // prompt for disconnect
+        nf.Dialog.showYesNoDialog({
+            dialogContent: 'Remove \'' + formatNodeAddress(node) + '\' from the cluster?',
+            overlayBackground: false,
+            yesHandler: function () {
+                remove(node.nodeId);
+            }
+        });
+    };
+
+    /**
      * Disconnects the node in the specified row.
      * 
      * @argument {string} nodeId     The node id
@@ -230,6 +278,86 @@ nf.ClusterTable = (function () {
         // perform the filter
         return item[args.property].search(filterExp) >= 0;
     };
+    
+    /**
+     * Show the node details.
+     * 
+     * @argument {object} item     The item
+     */
+    var showNodeDetails = function (item) {
+        $.ajax({
+            type: 'GET',
+            url: config.urls.nodes + '/' + encodeURIComponent(item.nodeId),
+            dataType: 'json'
+        }).done(function (response) {
+            var node = response.node;
+
+            // update the dialog fields
+            $('#node-id').text(node.nodeId);
+            $('#node-address').text(formatNodeAddress(node));
+
+            // format the events
+            var events = $('#node-events');
+            if ($.isArray(node.events) && node.events.length > 0) {
+                var eventMessages = [];
+                $.each(node.events, function (i, event) {
+                    eventMessages.push(event.timestamp + ": " + event.message);
+                });
+                $('<div></div>').append(nf.Common.formatUnorderedList(eventMessages)).appendTo(events);
+            } else {
+                events.append('<div><span class="unset">None</span></div>');
+            }
+
+            // show the dialog
+            $('#node-details-dialog').modal('show');
+        }).fail(nf.Common.handleAjaxError);
+    };
+    
+    /**
+     * Makes the specified node the primary node of the cluster.
+     * 
+     * @argument {object} item     The node item
+     */
+    var makePrimary = function (item) {
+        $.ajax({
+            type: 'PUT',
+            url: config.urls.nodes + '/' + encodeURIComponent(item.nodeId),
+            data: {
+                primary: true
+            },
+            dataType: 'json'
+        }).done(function (response) {
+            var grid = $('#cluster-table').data('gridInstance');
+            var data = grid.getData();
+
+            var node = response.node;
+
+            // start the update
+            data.beginUpdate();
+            data.updateItem(node.nodeId, node);
+
+            // need to find the previous primary node
+            // get the property grid data
+            var clusterItems = data.getItems();
+            $.each(clusterItems, function (i, otherNode) {
+                // attempt to identify the previous primary node
+                if (node.nodeId !== otherNode.nodeId && otherNode.primary === true) {
+                    // reset its primary status
+                    otherNode.primary = false;
+                    otherNode.status = 'CONNECTED';
+
+                    // set the new node state
+                    data.updateItem(otherNode.nodeId, otherNode);
+
+                    // no need to continue processing
+                    return false;
+                }
+            });
+
+            // end the update
+            data.endUpdate();
+        }).fail(nf.Common.handleAjaxError);
+    };
 
     return {
         /**
@@ -292,7 +420,7 @@ nf.ClusterTable = (function () {
 
             // define a custom formatter for the more details column
             var moreDetailsFormatter = function (row, cell, value, columnDef, dataContext) {
-                return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 4px;" onclick="javascript:nf.ClusterTable.showNodeDetails(\'' + row + '\');"/>';
+                return '<img src="images/iconDetails.png" title="View Details" class="pointer show-node-details" style="margin-top: 4px;"/>';
             };
 
             // define a custom formatter for the run status column
@@ -348,11 +476,11 @@ nf.ClusterTable = (function () {
 
                     // return the appropriate markup
                     if (canConnect) {
-                        return '<img src="images/iconConnect.png" title="Connect" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.ClusterTable.promptForConnect(\'' + row + '\');"/>&nbsp;<img src="images/iconDelete.png" title="Remove" class="pointer" onclick="javascript:nf.ClusterTable.promptForRemoval(\'' + row + '\');"/>';
+                        return '<img src="images/iconConnect.png" title="Connect" class="pointer prompt-for-connect" style="margin-top: 2px;"/>&nbsp;<img src="images/iconDelete.png" title="Remove" class="pointer prompt-for-removal"/>';
                     } else if (canDisconnect) {
-                        var actions = '<img src="images/iconDisconnect.png" title="Disconnect" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.ClusterTable.promptForDisconnect(\'' + row + '\');"/>';
+                        var actions = '<img src="images/iconDisconnect.png" title="Disconnect" class="pointer prompt-for-disconnect" style="margin-top: 2px;"/>';
                         if (canBecomePrimary) {
-                            actions += '&nbsp;<img src="images/iconPrimary.png" title="Make Primary" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.ClusterTable.makePrimary(\'' + row + '\');"/>';
+                            actions += '&nbsp;<img src="images/iconPrimary.png" title="Make Primary" class="pointer make-primary" style="margin-top: 2px;"/>';
                         }
                         return actions;
                     } else {
@@ -360,7 +488,7 @@ nf.ClusterTable = (function () {
                     }
                 };
 
-                columnModel.push({id: 'action', label: '&nbsp;', formatter: actionFormatter, resizable: false, sortable: false, width: 80, maxWidth: 80});
+                columnModel.push({id: 'actions', label: '&nbsp;', formatter: actionFormatter, resizable: false, sortable: false, width: 80, maxWidth: 80});
             }
 
             var clusterOptions = {
@@ -398,6 +526,31 @@ nf.ClusterTable = (function () {
                     sortAsc: args.sortAsc
                 }, clusterData);
             });
+            
+            // configure a click listener
+            clusterGrid.onClick.subscribe(function (e, args) {
+                var target = $(e.target);
+
+                // get the node at this row
+                var item = clusterData.getItem(args.row);
+
+                // determine the desired action
+                if (clusterGrid.getColumns()[args.cell].id === 'actions') {
+                    if (target.hasClass('prompt-for-connect')) {
+                        promptForConnect(item);
+                    } else if (target.hasClass('prompt-for-removal')) {
+                        promptForRemoval(item);
+                    } else if (target.hasClass('prompt-for-disconnect')) {
+                        promptForDisconnect(item);
+                    } else if (target.hasClass('make-primary')) {
+                        makePrimary(item);
+                    }
+                } else if (clusterGrid.getColumns()[args.cell].id === 'moreDetails') {
+                    if (target.hasClass('show-node-details')) {
+                        showNodeDetails(item);
+                    }
+                }
+            });
 
             // wire up the dataview to the grid
             clusterData.onRowCountChanged.subscribe(function (e, args) {
@@ -420,122 +573,6 @@ nf.ClusterTable = (function () {
         },
         
         /**
-         * Prompts to verify node connection.
-         * 
-         * @argument {string} row     The row
-         */
-        promptForConnect: function (row) {
-            var grid = $('#cluster-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var node = data.getItem(row);
-
-                // prompt to connect
-                nf.Dialog.showYesNoDialog({
-                    dialogContent: 'Connect \'' + formatNodeAddress(node) + '\' to this cluster?',
-                    overlayBackground: false,
-                    yesHandler: function () {
-                        connect(node.nodeId);
-                    }
-                });
-            }
-
-        },
-        
-        /**
-         * Prompts to verify node disconnection.
-         * 
-         * @argument {string} row     The row
-         */
-        promptForDisconnect: function (row) {
-            var grid = $('#cluster-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var node = data.getItem(row);
-
-                // prompt for disconnect
-                nf.Dialog.showYesNoDialog({
-                    dialogContent: 'Disconnect \'' + formatNodeAddress(node) + '\' from the cluster?',
-                    overlayBackground: false,
-                    yesHandler: function () {
-                        disconnect(node.nodeId);
-                    }
-                });
-            }
-        },
-        
-        /**
-         * Makes the specified node the primary node of the cluster.
-         * 
-         * @argument {string} row     The row
-         */
-        makePrimary: function (row) {
-            var grid = $('#cluster-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                $.ajax({
-                    type: 'PUT',
-                    url: config.urls.nodes + '/' + encodeURIComponent(item.nodeId),
-                    data: {
-                        primary: true
-                    },
-                    dataType: 'json'
-                }).done(function (response) {
-                    var node = response.node;
-
-                    // start the update
-                    data.beginUpdate();
-                    data.updateItem(node.nodeId, node);
-
-                    // need to find the previous primary node
-                    // get the property grid data
-                    var clusterItems = data.getItems();
-                    $.each(clusterItems, function (i, otherNode) {
-                        // attempt to identify the previous primary node
-                        if (node.nodeId !== otherNode.nodeId && otherNode.primary === true) {
-                            // reset its primary status
-                            otherNode.primary = false;
-                            otherNode.status = 'CONNECTED';
-
-                            // set the new node state
-                            data.updateItem(otherNode.nodeId, otherNode);
-
-                            // no need to continue processing
-                            return false;
-                        }
-                    });
-
-                    // end the update
-                    data.endUpdate();
-                }).fail(nf.Common.handleAjaxError);
-            }
-        },
-        
-        /**
-         * Prompts to verify node disconnection.
-         * 
-         * @argument {string} row     The row
-         */
-        promptForRemoval: function (row) {
-            var grid = $('#cluster-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var node = data.getItem(row);
-
-                // prompt for disconnect
-                nf.Dialog.showYesNoDialog({
-                    dialogContent: 'Remove \'' + formatNodeAddress(node) + '\' from the cluster?',
-                    overlayBackground: false,
-                    yesHandler: function () {
-                        remove(node.nodeId);
-                    }
-                });
-            }
-        },
-        
-        /**
          * Update the size of the grid based on its container's current size.
          */
         resetTableSize: function () {
@@ -575,46 +612,6 @@ nf.ClusterTable = (function () {
                     $('#total-nodes').text('0');
                 }
             }).fail(nf.Common.handleAjaxError);
-        },
-        
-        /**
-         * Populate the expanded row.
-         * 
-         * @argument {string} row     The row
-         */
-        showNodeDetails: function (row) {
-            var grid = $('#cluster-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                $.ajax({
-                    type: 'GET',
-                    url: config.urls.nodes + '/' + encodeURIComponent(item.nodeId),
-                    dataType: 'json'
-                }).done(function (response) {
-                    var node = response.node;
-
-                    // update the dialog fields
-                    $('#node-id').text(node.nodeId);
-                    $('#node-address').text(formatNodeAddress(node));
-
-                    // format the events
-                    var events = $('#node-events');
-                    if ($.isArray(node.events) && node.events.length > 0) {
-                        var eventMessages = [];
-                        $.each(node.events, function (i, event) {
-                            eventMessages.push(event.timestamp + ": " + event.message);
-                        });
-                        $('<div></div>').append(nf.Common.formatUnorderedList(eventMessages)).appendTo(events);
-                    } else {
-                        events.append('<div><span class="unset">None</span></div>');
-                    }
-
-                    // show the dialog
-                    $('#node-details-dialog').modal('show');
-                }).fail(nf.Common.handleAjaxError);
-            }
         }
     };
 }());
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9cd9d126/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/counters/nf-counters-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/counters/nf-counters-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/counters/nf-counters-table.js
index 536ffdb..2b26aaa 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/counters/nf-counters-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/counters/nf-counters-table.js
@@ -110,6 +110,26 @@ nf.CountersTable = (function () {
         // perform the filter
         return item[args.property].search(filterExp) >= 0;
     };
+    
+    /**
+     * Resets the specified counter.
+     * 
+     * @argument {object} item     The counter item
+     */
+    var resetCounter = function (item) {
+        $.ajax({
+            type: 'PUT',
+            url: config.urls.counters + '/' + encodeURIComponent(item.id),
+            dataType: 'json'
+        }).done(function (response) {
+            var counter = response.counter;
+
+            // get the table and update the row accordingly
+            var countersGrid = $('#counters-table').data('gridInstance');
+            var countersData = countersGrid.getData();
+            countersData.updateItem(counter.id, counter);
+        }).fail(nf.Common.handleAjaxError);
+    };
 
     return {
         /**
@@ -159,7 +179,7 @@ nf.CountersTable = (function () {
             if (nf.Common.isDFM()) {
                 // function for formatting the actions column
                 var actionFormatter = function (row, cell, value, columnDef, dataContext) {
-                    return '<img src="images/iconResetCounter.png" title="Reset" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.CountersTable.resetCounter(\'' + row + '\');"/>';
+                    return '<img src="images/iconResetCounter.png" title="Reset" class="pointer reset-counter" style="margin-top: 2px;"/>';
                 };
 
                 // add the action column
@@ -202,6 +222,21 @@ nf.CountersTable = (function () {
                     sortAsc: args.sortAsc
                 }, countersData);
             });
+            
+            // configure a click listener
+            countersGrid.onClick.subscribe(function (e, args) {
+                var target = $(e.target);
+
+                // get the node at this row
+                var item = countersData.getItem(args.row);
+
+                // determine the desired action
+                if (countersGrid.getColumns()[args.cell].id === 'actions') {
+                    if (target.hasClass('reset-counter')) {
+                        resetCounter(item);
+                    }
+                }
+            });
 
             // wire up the dataview to the grid
             countersData.onRowCountChanged.subscribe(function (e, args) {
@@ -224,32 +259,6 @@ nf.CountersTable = (function () {
         },
         
         /**
-         * Resets the specified counter.
-         * 
-         * @argument {string} row     The row
-         */
-        resetCounter: function (row) {
-            var grid = $('#counters-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                $.ajax({
-                    type: 'PUT',
-                    url: config.urls.counters + '/' + encodeURIComponent(item.id),
-                    dataType: 'json'
-                }).done(function (response) {
-                    var counter = response.counter;
-
-                    // get the table and update the row accordingly
-                    var countersGrid = $('#counters-table').data('gridInstance');
-                    var countersData = countersGrid.getData();
-                    countersData.updateItem(counter.id, counter);
-                }).fail(nf.Common.handleAjaxError);
-            }
-        },
-        
-        /**
          * Update the size of the grid based on its container's current size.
          */
         resetTableSize: function () {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9cd9d126/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
index be0ea73..72fc549 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
@@ -244,7 +244,7 @@ nf.HistoryTable = (function () {
 
         // define a custom formatter for the more details column
         var moreDetailsFormatter = function (row, cell, value, columnDef, dataContext) {
-            return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 4px;" onclick="javascript:nf.HistoryTable.showActionDetails(\'' + row + '\');"/>';
+            return '<img src="images/iconDetails.png" title="View Details" class="pointer show-action-details" style="margin-top: 4px;"/>';
         };
 
         // initialize the templates table
@@ -283,6 +283,21 @@ nf.HistoryTable = (function () {
         });
         historyGrid.setSortColumn('timestamp', false);
 
+        // configure a click listener
+        historyGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = historyModel.getItem(args.row);
+
+            // determine the desired action
+            if (historyGrid.getColumns()[args.cell].id === 'moreDetails') {
+                if (target.hasClass('show-action-details')) {
+                    showActionDetails(item);
+                }
+            }
+        });
+
         // listen for when the viewport changes so we can fetch the appropriate records
         historyGrid.onViewportChanged.subscribe(function (e, args) {
             var vp = historyGrid.getViewport();
@@ -325,6 +340,68 @@ nf.HistoryTable = (function () {
             nf.HistoryTable.loadHistoryTable();
         }).fail(nf.Common.handleAjaxError);
     };
+    
+    /**
+     * Shows the details for the specified action.
+     * 
+     * @param {object} action
+     */
+    var showActionDetails = function (action) {
+        // create the markup for the dialog
+        var detailsMarkup = $('<div></div>').append(
+                $('<div class="action-detail"><div class="history-details-name">Id</div>' + nf.Common.escapeHtml(action.sourceId) + '</div>'));
+
+        // get any component details
+        var componentDetails = action.componentDetails;
+
+        // inspect the operation to determine if there are any component details
+        if (nf.Common.isDefinedAndNotNull(componentDetails)) {
+            if (action.sourceType === 'Processor') {
+                detailsMarkup.append(
+                        $('<div class="action-detail"><div class="history-details-name">Type</div>' + nf.Common.escapeHtml(componentDetails.type) + '</div>'));
+            } else if (action.sourceType === 'RemoteProcessGroup') {
+                detailsMarkup.append(
+                        $('<div class="action-detail"><div class="history-details-name">Uri</div>' + nf.Common.formatValue(componentDetails.uri) + '</div>'));
+            }
+        }
+
+        // get any action details
+        var actionDetails = action.actionDetails;
+
+        // inspect the operation to determine if there are any action details
+        if (nf.Common.isDefinedAndNotNull(actionDetails)) {
+            if (action.operation === 'Configure') {
+                detailsMarkup.append(
+                        $('<div class="action-detail"><div class="history-details-name">Name</div>' + nf.Common.formatValue(actionDetails.name) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Value</div>' + nf.Common.formatValue(actionDetails.value) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Previous Value</div>' + nf.Common.formatValue(actionDetails.previousValue) + '</div>'));
+            } else if (action.operation === 'Connect' || action.operation === 'Disconnect') {
+                detailsMarkup.append(
+                        $('<div class="action-detail"><div class="history-details-name">Source Id</div>' + nf.Common.escapeHtml(actionDetails.sourceId) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Source Name</div>' + nf.Common.formatValue(actionDetails.sourceName) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Source Type</div>' + nf.Common.escapeHtml(actionDetails.sourceType) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Relationship(s)</div>' + nf.Common.formatValue(actionDetails.relationship) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Destination Id</div>' + nf.Common.escapeHtml(actionDetails.destinationId) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Destination Name</div>' + nf.Common.formatValue(actionDetails.destinationName) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Destination Type</div>' + nf.Common.escapeHtml(actionDetails.destinationType) + '</div>'));
+            } else if (action.operation === 'Move') {
+                detailsMarkup.append(
+                        $('<div class="action-detail"><div class="history-details-name">Group</div>' + nf.Common.formatValue(actionDetails.group) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Group Id</div>' + nf.Common.escapeHtml(actionDetails.groupId) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Previous Group</div>' + nf.Common.formatValue(actionDetails.previousGroup) + '</div>')).append(
+                        $('<div class="action-detail"><div class="history-details-name">Previous Group Id</div>' + nf.Common.escapeHtml(actionDetails.previousGroupId) + '</div>'));
+            } else if (action.operation === 'Purge') {
+                detailsMarkup.append(
+                        $('<div class="action-detail"><div class="history-details-name">End Date</div>' + nf.Common.escapeHtml(actionDetails.endDate) + '</div>'));
+            }
+        }
+
+        // populate the dialog
+        $('#action-details').append(detailsMarkup);
+
+        // show the dialog
+        $('#action-details-dialog').modal('show');
+    };
 
     return {
         init: function () {
@@ -356,74 +433,6 @@ nf.HistoryTable = (function () {
 
             // request refresh of the current 'page'
             historyGrid.onViewportChanged.notify();
-        },
-        
-        /**
-         * Shows the details for the specified action.
-         * 
-         * @param {object} index
-         */
-        showActionDetails: function (index) {
-            var historyGrid = $('#history-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(historyGrid)) {
-                var historyModel = historyGrid.getData();
-                var action = historyModel.getItem(index);
-
-                // create the markup for the dialog
-                var detailsMarkup = $('<div></div>').append(
-                        $('<div class="action-detail"><div class="history-details-name">Id</div>' + nf.Common.escapeHtml(action.sourceId) + '</div>'));
-
-                // get any component details
-                var componentDetails = action.componentDetails;
-
-                // inspect the operation to determine if there are any component details
-                if (nf.Common.isDefinedAndNotNull(componentDetails)) {
-                    if (action.sourceType === 'Processor') {
-                        detailsMarkup.append(
-                                $('<div class="action-detail"><div class="history-details-name">Type</div>' + nf.Common.escapeHtml(componentDetails.type) + '</div>'));
-                    } else if (action.sourceType === 'RemoteProcessGroup') {
-                        detailsMarkup.append(
-                                $('<div class="action-detail"><div class="history-details-name">Uri</div>' + nf.Common.formatValue(componentDetails.uri) + '</div>'));
-                    }
-                }
-
-                // get any action details
-                var actionDetails = action.actionDetails;
-
-                // inspect the operation to determine if there are any action details
-                if (nf.Common.isDefinedAndNotNull(actionDetails)) {
-                    if (action.operation === 'Configure') {
-                        detailsMarkup.append(
-                                $('<div class="action-detail"><div class="history-details-name">Name</div>' + nf.Common.formatValue(actionDetails.name) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Value</div>' + nf.Common.formatValue(actionDetails.value) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Previous Value</div>' + nf.Common.formatValue(actionDetails.previousValue) + '</div>'));
-                    } else if (action.operation === 'Connect' || action.operation === 'Disconnect') {
-                        detailsMarkup.append(
-                                $('<div class="action-detail"><div class="history-details-name">Source Id</div>' + nf.Common.escapeHtml(actionDetails.sourceId) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Source Name</div>' + nf.Common.formatValue(actionDetails.sourceName) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Source Type</div>' + nf.Common.escapeHtml(actionDetails.sourceType) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Relationship(s)</div>' + nf.Common.formatValue(actionDetails.relationship) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Destination Id</div>' + nf.Common.escapeHtml(actionDetails.destinationId) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Destination Name</div>' + nf.Common.formatValue(actionDetails.destinationName) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Destination Type</div>' + nf.Common.escapeHtml(actionDetails.destinationType) + '</div>'));
-                    } else if (action.operation === 'Move') {
-                        detailsMarkup.append(
-                                $('<div class="action-detail"><div class="history-details-name">Group</div>' + nf.Common.formatValue(actionDetails.group) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Group Id</div>' + nf.Common.escapeHtml(actionDetails.groupId) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Previous Group</div>' + nf.Common.formatValue(actionDetails.previousGroup) + '</div>')).append(
-                                $('<div class="action-detail"><div class="history-details-name">Previous Group Id</div>' + nf.Common.escapeHtml(actionDetails.previousGroupId) + '</div>'));
-                    } else if (action.operation === 'Purge') {
-                        detailsMarkup.append(
-                                $('<div class="action-detail"><div class="history-details-name">End Date</div>' + nf.Common.escapeHtml(actionDetails.endDate) + '</div>'));
-                    }
-                }
-
-                // populate the dialog
-                $('#action-details').append(detailsMarkup);
-
-                // show the dialog
-                $('#action-details-dialog').modal('show');
-            }
         }
     };
 }());
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9cd9d126/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
index 759bcda..2880375 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
@@ -553,7 +553,7 @@ nf.ProvenanceTable = (function () {
 
         // define a custom formatter for the more details column
         var moreDetailsFormatter = function (row, cell, value, columnDef, dataContext) {
-            return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 4px;" onclick="javascript:nf.ProvenanceTable.showEventDetailsByIndex(\'' + row + '\');"/>';
+            return '<img src="images/iconDetails.png" title="View Details" class="pointer show-event-details" style="margin-top: 4px;"/>';
         };
 
         // define how general values are formatted
@@ -570,12 +570,12 @@ nf.ProvenanceTable = (function () {
 
             // conditionally include the cluster node id
             if (nf.Common.SUPPORTS_SVG) {
-                markup += '<img src="images/iconLineage.png" title="Show Lineage" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.ProvenanceTable.showLineage(\'' + row + '\');"/>';
+                markup += '<img src="images/iconLineage.png" title="Show Lineage" class="pointer show-lineage" style="margin-top: 2px;"/>';
             }
 
             // conditionally support going to the component
             if (isInShell && nf.Common.isDefinedAndNotNull(dataContext.groupId)) {
-                markup += '&nbsp;<img src="images/iconGoTo.png" title="Go To" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.ProvenanceTable.goTo(\'' + row + '\');"/>';
+                markup += '&nbsp;<img src="images/iconGoTo.png" title="Go To" class="pointer go-to" style="margin-top: 2px;"/>';
             }
 
             return markup;
@@ -599,7 +599,7 @@ nf.ProvenanceTable = (function () {
 
         // conditionally show the action column
         if (nf.Common.SUPPORTS_SVG || isInShell) {
-            provenanceColumns.push({id: 'action', name: '&nbsp;', formatter: showLineageFormatter, resizable: false, sortable: false, width: 50, maxWidth: 50});
+            provenanceColumns.push({id: 'actions', name: '&nbsp;', formatter: showLineageFormatter, resizable: false, sortable: false, width: 50, maxWidth: 50});
         }
 
         var provenanceOptions = {
@@ -641,6 +641,27 @@ nf.ProvenanceTable = (function () {
                 sortAsc: args.sortAsc
             }, provenanceData);
         });
+        
+        // configure a click listener
+        provenanceGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = provenanceData.getItem(args.row);
+
+            // determine the desired action
+            if (provenanceGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('show-lineage')) {
+                    nf.ProvenanceLineage.showLineage(item.flowFileUuid, item.eventId.toString(), item.clusterNodeId);
+                } else if (target.hasClass('go-to')) {
+                    goTo(item);
+                }
+            } else if (provenanceGrid.getColumns()[args.cell].id === 'moreDetails') {
+                if (target.hasClass('show-event-details')) {
+                    nf.ProvenanceTable.showEventDetails(item);
+                }
+            }
+        });
 
         // wire up the dataview to the grid
         provenanceData.onRowCountChanged.subscribe(function (e, args) {
@@ -884,6 +905,25 @@ nf.ProvenanceTable = (function () {
         }
     };
 
+    /**
+     * Goes to the specified component if possible.
+     * 
+     * @argument {object} item       The event it
+     */
+    var goTo = function (item) {
+        // ensure the component is still present in the flow
+        if (nf.Common.isDefinedAndNotNull(item.groupId)) {
+            // only attempt this if we're within a frame
+            if (top !== window) {
+                // and our parent has canvas utils and shell defined
+                if (nf.Common.isDefinedAndNotNull(parent.nf) && nf.Common.isDefinedAndNotNull(parent.nf.CanvasUtils) && nf.Common.isDefinedAndNotNull(parent.nf.Shell)) {
+                    parent.nf.CanvasUtils.showComponent(item.groupId, item.componentId);
+                    parent.$('#shell-close-button').click();
+                }
+            }
+        }
+    };
+
     return {
         /**
          * The max delay between requests.
@@ -910,31 +950,6 @@ nf.ProvenanceTable = (function () {
         },
         
         /**
-         * Goes to the specified component if possible.
-         * 
-         * @argument {string} row       The row
-         */
-        goTo: function (row) {
-            var grid = $('#provenance-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // ensure the component is still present in the flow
-                if (nf.Common.isDefinedAndNotNull(item.groupId)) {
-                    // only attempt this if we're within a frame
-                    if (top !== window) {
-                        // and our parent has canvas utils and shell defined
-                        if (nf.Common.isDefinedAndNotNull(parent.nf) && nf.Common.isDefinedAndNotNull(parent.nf.CanvasUtils) && nf.Common.isDefinedAndNotNull(parent.nf.Shell)) {
-                            parent.nf.CanvasUtils.showComponent(item.groupId, item.componentId);
-                            parent.$('#shell-close-button').click();
-                        }
-                    }
-                }
-            }
-        },
-        
-        /**
          * Update the size of the grid based on its container's current size.
          */
         resetTableSize: function () {
@@ -1096,36 +1111,6 @@ nf.ProvenanceTable = (function () {
         },
         
         /**
-         * Shows the lineage for the event in the specified row.
-         * 
-         * @param {type} row
-         */
-        showLineage: function (row) {
-            var grid = $('#provenance-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.ProvenanceLineage.showLineage(item.flowFileUuid, item.eventId.toString(), item.clusterNodeId);
-            }
-        },
-        
-        /**
-         * Gets the event details and shows the details dialog.
-         * 
-         * @param {long} index
-         */
-        showEventDetailsByIndex: function (index) {
-            var provenanceGrid = $('#provenance-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(provenanceGrid)) {
-                var provenanceModel = provenanceGrid.getData();
-                var event = provenanceModel.getItem(index);
-
-                // show the event details
-                nf.ProvenanceTable.showEventDetails(event);
-            }
-        },
-        
-        /**
          * Shows the details for the specified action.
          * 
          * @param {object} event

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9cd9d126/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
index 1756207..ec8f49e 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
@@ -54,6 +54,22 @@ nf.TemplatesTable = (function () {
     };
 
     /**
+     * Prompts the user before attempting to delete the specified template.
+     * 
+     * @argument {object} template     The template
+     */
+    var promptToDeleteTemplate = function (template) {
+        // prompt for deletion
+        nf.Dialog.showYesNoDialog({
+            dialogContent: 'Delete template \'' + nf.Common.escapeHtml(template.name) + '\'?',
+            overlayBackground: false,
+            yesHandler: function () {
+                deleteTemplate(template.id);
+            }
+        });
+    };
+
+    /**
      * Deletes the template with the specified id.
      * 
      * @argument {string} templateId     The template id
@@ -177,11 +193,11 @@ nf.TemplatesTable = (function () {
 
             // function for formatting the actions column
             var actionFormatter = function (row, cell, value, columnDef, dataContext) {
-                var markup = '<img src="images/iconExport.png" title="Download" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.TemplatesTable.exportTemplate(\'' + row + '\');"/>';
+                var markup = '<img src="images/iconExport.png" title="Download" class="pointer export-template" style="margin-top: 2px;"/>';
 
                 // all DFMs to remove templates
                 if (nf.Common.isDFM()) {
-                    markup += '&nbsp;<img src="images/iconDelete.png" title="Remove Template" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.TemplatesTable.promptToDeleteTemplate(\'' + row + '\');"/>';
+                    markup += '&nbsp;<img src="images/iconDelete.png" title="Remove Template" class="pointer prompt-to-delete-template" style="margin-top: 2px;"/>';
                 }
                 return markup;
             };
@@ -230,6 +246,23 @@ nf.TemplatesTable = (function () {
                 }, templatesData);
             });
 
+            // configure a click listener
+            templatesGrid.onClick.subscribe(function (e, args) {
+                var target = $(e.target);
+
+                // get the node at this row
+                var item = templatesData.getItem(args.row);
+
+                // determine the desired action
+                if (templatesGrid.getColumns()[args.cell].id === 'actions') {
+                    if (target.hasClass('export-template')) {
+                        window.open(config.urls.templates + '/' + encodeURIComponent(item.id));
+                    } else if (target.hasClass('prompt-to-delete-template')) {
+                        promptToDeleteTemplate(item);
+                    }
+                }
+            });
+
             // wire up the dataview to the grid
             templatesData.onRowCountChanged.subscribe(function (e, args) {
                 templatesGrid.updateRowCount();
@@ -261,42 +294,6 @@ nf.TemplatesTable = (function () {
         },
         
         /**
-         * Exports the specified template.
-         * 
-         * @argument {string} row     The row
-         */
-        exportTemplate: function (row) {
-            var grid = $('#templates-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                window.open(config.urls.templates + '/' + encodeURIComponent(item.id));
-            }
-        },
-        
-        /**
-         * Prompts the user before attempting to delete the specified template.
-         * 
-         * @argument {string} row     The row
-         */
-        promptToDeleteTemplate: function (row) {
-            var grid = $('#templates-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var template = data.getItem(row);
-
-                // prompt for deletion
-                nf.Dialog.showYesNoDialog({
-                    dialogContent: 'Delete template \'' + nf.Common.escapeHtml(template.name) + '\'?',
-                    overlayBackground: false,
-                    yesHandler: function () {
-                        deleteTemplate(template.id);
-                    }
-                });
-            }
-        },
-        
-        /**
          * Load the processor templates table.
          */
         loadTemplatesTable: function () {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9cd9d126/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/users/nf-users-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/users/nf-users-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/users/nf-users-table.js
index 996544f..88ea225 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/users/nf-users-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/users/nf-users-table.js
@@ -483,7 +483,7 @@ nf.UsersTable = (function () {
 
         // define a custom formatter for the more details column
         var moreDetailsFormatter = function (row, cell, value, columnDef, dataContext) {
-            return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 4px;" onclick="javascript:nf.UsersTable.showUserDetails(\'' + row + '\');"/>';
+            return '<img src="images/iconDetails.png" title="View Details" class="pointer show-user-details" style="margin-top: 4px;"/>';
         };
 
         // function for formatting the last accessed time
@@ -566,20 +566,20 @@ nf.UsersTable = (function () {
 
             // if this represents a grouped row
             if (nf.Common.isDefinedAndNotNull(dataContext.userGroup) && grouped) {
-                var actions = '<img src="images/iconEdit.png" title="Edit Access" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.UsersTable.updateGroupAccess(\'' + row + '\');"/>&nbsp;<img src="images/iconRevoke.png" title="Revoke Access" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.UsersTable.revokeGroupAccess(\'' + row + '\');"/>&nbsp;&nbsp;<img src="images/ungroup.png" title="Ungroup" class="pointer" onclick="javascript:nf.UsersTable.ungroup(\'' + row + '\');"/>';
+                var actions = '<img src="images/iconEdit.png" title="Edit Access" class="pointer update-group-access" style="margin-top: 2px;"/>&nbsp;<img src="images/iconRevoke.png" title="Revoke Access" class="pointer revoke-group-access" style="margin-top: 2px;"/>&nbsp;&nbsp;<img src="images/ungroup.png" title="Ungroup" class="pointer ungroup"/>';
             } else {
                 // return the appropriate markup for an individual user
-                var actions = '<img src="images/iconEdit.png" title="Edit Access" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.UsersTable.updateUserAccess(\'' + row + '\');"/>';
+                var actions = '<img src="images/iconEdit.png" title="Edit Access" class="pointer update-user-access" style="margin-top: 2px;"/>';
 
                 if (dataContext.status === 'ACTIVE') {
-                    actions += '&nbsp;<img src="images/iconRevoke.png" title="Revoke Access" class="pointer" onclick="javascript:nf.UsersTable.revokeUserAccess(\'' + row + '\');"/>';
+                    actions += '&nbsp;<img src="images/iconRevoke.png" title="Revoke Access" class="pointer revoke-user-access"/>';
 
                     // add an ungroup active if appropriate
                     if (nf.Common.isDefinedAndNotNull(dataContext.userGroup)) {
-                        actions += '&nbsp;&nbsp;<img src="images/ungroup.png" title="Ungroup" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.UsersTable.ungroupUser(\'' + row + '\');"/>';
+                        actions += '&nbsp;&nbsp;<img src="images/ungroup.png" title="Ungroup" class="pointer ungroup-user" style="margin-top: 2px;"/>';
                     }
                 } else {
-                    actions += '&nbsp;<img src="images/iconDelete.png" title="Delete Account" class="pointer" onclick="javascript:nf.UsersTable.deleteUserAccount(\'' + row + '\');"/>';
+                    actions += '&nbsp;<img src="images/iconDelete.png" title="Delete Account" class="pointer delete-user-account"/>';
                 }
             }
 
@@ -632,6 +632,37 @@ nf.UsersTable = (function () {
                 sortAsc: args.sortAsc
             }, usersData);
         });
+        
+        // configure a click listener
+        usersGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = usersData.getItem(args.row);
+
+            // determine the desired action
+            if (usersGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('update-group-access')) {
+                    updateGroupAccess(item);
+                } else if (target.hasClass('revoke-group-access')) {
+                    revokeGroupAccess(item);
+                } else if (target.hasClass('ungroup')) {
+                    ungroup(item);
+                } else if (target.hasClass('update-user-access')) {
+                    updateUserAccess(item);
+                } else if (target.hasClass('revoke-user-access')) {
+                    revokeUserAccess(item);
+                } else if (target.hasClass('ungroup-user')) {
+                    ungroupUser(item);
+                } else if (target.hasClass('delete-user-account')) {
+                    deleteUserAccount(item);
+                }
+            } else if (usersGrid.getColumns()[args.cell].id === 'moreDetails') {
+                if (target.hasClass('show-user-details')) {
+                    showUserDetails(item);
+                }
+            }
+        });
 
         // wire up the dataview to the grid
         usersData.onRowCountChanged.subscribe(function (e, args) {
@@ -804,193 +835,192 @@ nf.UsersTable = (function () {
         }
     };
 
-    return {
-        init: function () {
-            initUserDetailsDialog();
-            initUserRolesDialog();
-            initGroupRolesDialog();
-            initUserRevokeDialog();
-            initUserDeleteDialog();
-            initUserGroupDialog();
-            initGroupRevokeDialog();
-            initUsersTable();
-        },
-        
-        /**
-         * Disables the specified user's account.
-         * 
-         * @argument {string} row        The row
-         */
-        revokeUserAccess: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
+    /**
+     * Shows details for the specified user.
+     * 
+     * @param {object} user
+     */
+    var showUserDetails = function (user) {
+        var grouped = $('#group-collaspe-checkbox').hasClass('checkbox-checked');
+
+        // update the dialog fields
+        $('#user-name-details-dialog').text(user.userName);
+        $('#user-dn-details-dialog').text(user.dn);
+
+        // handle fields that could vary for groups
+        if (nf.Common.isDefinedAndNotNull(user.creation)) {
+            $('#user-created-details-dialog').text(user.creation);
+        } else if (grouped && nf.Common.isDefinedAndNotNull(user.userGroup)) {
+            $('#user-created-details-dialog').html('<span class="unset">Multiple users with different creation timestamps.</span>');
+        } else {
+            $('#user-created-details-dialog').html('<span class="unset">No creation timestamp set</span>');
+        }
 
-                // populate the users info
-                $('#user-id-revoke-dialog').val(item.id);
-                $('#user-name-revoke-dialog').text(item.userName);
+        if (nf.Common.isDefinedAndNotNull(user.lastVerified)) {
+            $('#user-verified-details-dialog').text(user.lastVerified);
+        } else if (grouped && nf.Common.isDefinedAndNotNull(user.userGroup)) {
+            $('#user-verified-details-dialog').html('<span class="unset">Multiple users with different last verified timestamps.</span>');
+        } else {
+            $('#user-verified-details-dialog').html('<span class="unset">No last verified timestamp set.</span>');
+        }
 
-                // show the dialog
-                $('#user-revoke-dialog').modal('show');
-            }
-        },
-        
-        /**
-         * Delete's the specified user's account.
-         * 
-         * @argument {string} row        The row
-         */
-        deleteUserAccount: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
+        if (nf.Common.isDefinedAndNotNull(user.justification)) {
+            $('#user-justification-details-dialog').text(user.justification);
+        } else if (grouped && nf.Common.isDefinedAndNotNull(user.userGroup)) {
+            $('#user-justification-details-dialog').html('<span class="unset">Multiple users with different justifications.</span>');
+        } else {
+            $('#user-justification-details-dialog').html('<span class="unset">No justification set.</span>');
+        }
 
-                // populate the users info
-                $('#user-id-delete-dialog').val(item.id);
-                $('#user-name-delete-dialog').text(item.userName);
+        // show the dialog
+        $('#user-details-dialog').modal('show');
+    };
+    
+    /**
+     * Updates the specified groups level of access.
+     * 
+     * @argument {object} item        The user item
+     */
+    var updateGroupAccess = function (item) {
+        // record the current group
+        $('#group-name-roles-dialog').text(item.userGroup);
 
-                // show the dialog
-                $('#user-delete-dialog').modal('show');
-            }
-        },
-        
-        /**
-         * Disables the specified group's account.
-         * 
-         * @argument {string} row        The row
-         */
-        revokeGroupAccess: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
+        // show the dialog
+        $('#group-roles-dialog').modal('show');
+    };
+    
+    /**
+     * Disables the specified group's account.
+     * 
+     * @argument {object} item        The user item
+     */
+    var revokeGroupAccess = function (item) {
+        // record the current group
+        $('#group-name-revoke-dialog').text(item.userGroup);
 
-                // record the current group
-                $('#group-name-revoke-dialog').text(item.userGroup);
+        // show the dialog
+        $('#group-revoke-dialog').modal('show');
+    };
 
-                // show the dialog
-                $('#group-revoke-dialog').modal('show');
+    /**
+     * Ungroups the specified group.
+     * 
+     * @argument {object} item        The user item
+     */
+    var ungroup = function (item) {
+        // prompt for ungroup
+        nf.Dialog.showYesNoDialog({
+            dialogContent: 'Remove all users from group \'' + nf.Common.escapeHtml(item.userGroup) + '\'?',
+            overlayBackground: false,
+            yesHandler: function () {
+                $.ajax({
+                    type: 'DELETE',
+                    url: config.urls.userGroups + '/' + encodeURIComponent(item.userGroup),
+                    dataType: 'json'
+                }).done(function (response) {
+                    nf.UsersTable.loadUsersTable();
+                }).fail(nf.Common.handleAjaxError);
             }
-        },
-        
-        /**
-         * Updates the specified users's level of access.
-         * 
-         * @argument {string} row        The row
-         */
-        updateUserAccess: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // populate the user info
-                $('#user-id-roles-dialog').val(item.id);
-                $('#user-name-roles-dialog').attr('title', item.dn).text(item.userName);
-                $('#user-justification-roles-dialog').html(nf.Common.formatValue(item.justification));
-
-                // function for checking a checkbox
-                var check = function (domId) {
-                    $('#' + domId).removeClass('checkbox-unchecked').addClass('checkbox-checked');
-                };
-
-                // go through each user role
-                $.each(item.authorities, function (i, authority) {
-                    if (authority === 'ROLE_ADMIN') {
-                        check('role-admin-checkbox');
-                    } else if (authority === 'ROLE_DFM') {
-                        check('role-dfm-checkbox');
-                    } else if (authority === 'ROLE_PROVENANCE') {
-                        check('role-provenance-checkbox');
-                    } else if (authority === 'ROLE_MONITOR') {
-                        check('role-monitor-checkbox');
-                    } else if (authority === 'ROLE_NIFI') {
-                        check('role-nifi-checkbox');
-                    } else if (authority === 'ROLE_PROXY') {
-                        check('role-proxy-checkbox');
-                    }
-                });
+        });
+    };
+    
+    /**
+     * Updates the specified users's level of access.
+     * 
+     * @argument {object} item        The user item
+     */
+    var updateUserAccess = function (item) {
+        // populate the user info
+        $('#user-id-roles-dialog').val(item.id);
+        $('#user-name-roles-dialog').attr('title', item.dn).text(item.userName);
+        $('#user-justification-roles-dialog').html(nf.Common.formatValue(item.justification));
+
+        // function for checking a checkbox
+        var check = function (domId) {
+            $('#' + domId).removeClass('checkbox-unchecked').addClass('checkbox-checked');
+        };
 
-                // show the dialog
-                $('#user-roles-dialog').modal('show');
+        // go through each user role
+        $.each(item.authorities, function (i, authority) {
+            if (authority === 'ROLE_ADMIN') {
+                check('role-admin-checkbox');
+            } else if (authority === 'ROLE_DFM') {
+                check('role-dfm-checkbox');
+            } else if (authority === 'ROLE_PROVENANCE') {
+                check('role-provenance-checkbox');
+            } else if (authority === 'ROLE_MONITOR') {
+                check('role-monitor-checkbox');
+            } else if (authority === 'ROLE_NIFI') {
+                check('role-nifi-checkbox');
+            } else if (authority === 'ROLE_PROXY') {
+                check('role-proxy-checkbox');
             }
-        },
-        
-        /**
-         * Updates the specified groups level of access.
-         * 
-         * @argument {string} row        The row
-         */
-        updateGroupAccess: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
+        });
 
-                // record the current group
-                $('#group-name-roles-dialog').text(item.userGroup);
+        // show the dialog
+        $('#user-roles-dialog').modal('show');
+    };
+    
+    /**
+     * Disables the specified user's account.
+     * 
+     * @argument {object} item        The user item
+     */
+    var revokeUserAccess = function (item) {
+        // populate the users info
+        $('#user-id-revoke-dialog').val(item.id);
+        $('#user-name-revoke-dialog').text(item.userName);
 
-                // show the dialog
-                $('#group-roles-dialog').modal('show');
-            }
-        },
-        
-        /**
-         * Prompts to verify group removal.
-         * 
-         * @argument {string} row        The row
-         */
-        ungroupUser: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // prompt for ungroup
-                nf.Dialog.showYesNoDialog({
-                    dialogContent: 'Remove user \'' + nf.Common.escapeHtml(item.userName) + '\' from group \'' + nf.Common.escapeHtml(item.userGroup) + '\'?',
-                    overlayBackground: false,
-                    yesHandler: function () {
-                        $.ajax({
-                            type: 'DELETE',
-                            url: config.urls.userGroups + '/' + encodeURIComponent(item.userGroup) + '/users/' + encodeURIComponent(item.id),
-                            dataType: 'json'
-                        }).done(function (response) {
-                            nf.UsersTable.loadUsersTable();
-                        }).fail(nf.Common.handleAjaxError);
-                    }
-                });
-            }
-        },
-        
-        /**
-         * Ungroups the specified group.
-         * 
-         * @argument {string} row        The row
-         */
-        ungroup: function (row) {
-            var grid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // prompt for ungroup
-                nf.Dialog.showYesNoDialog({
-                    dialogContent: 'Remove all users from group \'' + nf.Common.escapeHtml(item.userGroup) + '\'?',
-                    overlayBackground: false,
-                    yesHandler: function () {
-                        $.ajax({
-                            type: 'DELETE',
-                            url: config.urls.userGroups + '/' + encodeURIComponent(item.userGroup),
-                            dataType: 'json'
-                        }).done(function (response) {
-                            nf.UsersTable.loadUsersTable();
-                        }).fail(nf.Common.handleAjaxError);
-                    }
-                });
+        // show the dialog
+        $('#user-revoke-dialog').modal('show');
+    };
+    
+    /**
+     * Prompts to verify group removal.
+     * 
+     * @argument {object} item        The user item
+     */
+    var ungroupUser = function (item) {
+        // prompt for ungroup
+        nf.Dialog.showYesNoDialog({
+            dialogContent: 'Remove user \'' + nf.Common.escapeHtml(item.userName) + '\' from group \'' + nf.Common.escapeHtml(item.userGroup) + '\'?',
+            overlayBackground: false,
+            yesHandler: function () {
+                $.ajax({
+                    type: 'DELETE',
+                    url: config.urls.userGroups + '/' + encodeURIComponent(item.userGroup) + '/users/' + encodeURIComponent(item.id),
+                    dataType: 'json'
+                }).done(function (response) {
+                    nf.UsersTable.loadUsersTable();
+                }).fail(nf.Common.handleAjaxError);
             }
+        });
+    };
+
+    /**
+     * Delete's the specified user's account.
+     * 
+     * @argument {object} item        The user item
+     */
+    var deleteUserAccount = function (item) {
+        // populate the users info
+        $('#user-id-delete-dialog').val(item.id);
+        $('#user-name-delete-dialog').text(item.userName);
+
+        // show the dialog
+        $('#user-delete-dialog').modal('show');
+    };
+
+    return {
+        init: function () {
+            initUserDetailsDialog();
+            initUserRolesDialog();
+            initGroupRolesDialog();
+            initUserRevokeDialog();
+            initUserDeleteDialog();
+            initUserGroupDialog();
+            initGroupRevokeDialog();
+            initUsersTable();
         },
         
         /**
@@ -1037,54 +1067,6 @@ nf.UsersTable = (function () {
                     $('#total-users').text('0');
                 }
             }).fail(nf.Common.handleAjaxError);
-        },
-        
-        /**
-         * Shows details for the specified user.
-         * 
-         * @param {string} row
-         */
-        showUserDetails: function (row) {
-            var usersGrid = $('#users-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(usersGrid)) {
-                var usersData = usersGrid.getData();
-
-                // get the user
-                var user = usersData.getItem(row);
-                var grouped = $('#group-collaspe-checkbox').hasClass('checkbox-checked');
-
-                // update the dialog fields
-                $('#user-name-details-dialog').text(user.userName);
-                $('#user-dn-details-dialog').text(user.dn);
-
-                // handle fields that could vary for groups
-                if (nf.Common.isDefinedAndNotNull(user.creation)) {
-                    $('#user-created-details-dialog').text(user.creation);
-                } else if (grouped && nf.Common.isDefinedAndNotNull(user.userGroup)) {
-                    $('#user-created-details-dialog').html('<span class="unset">Multiple users with different creation timestamps.</span>');
-                } else {
-                    $('#user-created-details-dialog').html('<span class="unset">No creation timestamp set</span>');
-                }
-
-                if (nf.Common.isDefinedAndNotNull(user.lastVerified)) {
-                    $('#user-verified-details-dialog').text(user.lastVerified);
-                } else if (grouped && nf.Common.isDefinedAndNotNull(user.userGroup)) {
-                    $('#user-verified-details-dialog').html('<span class="unset">Multiple users with different last verified timestamps.</span>');
-                } else {
-                    $('#user-verified-details-dialog').html('<span class="unset">No last verified timestamp set.</span>');
-                }
-
-                if (nf.Common.isDefinedAndNotNull(user.justification)) {
-                    $('#user-justification-details-dialog').text(user.justification);
-                } else if (grouped && nf.Common.isDefinedAndNotNull(user.userGroup)) {
-                    $('#user-justification-details-dialog').html('<span class="unset">Multiple users with different justifications.</span>');
-                } else {
-                    $('#user-justification-details-dialog').html('<span class="unset">No justification set.</span>');
-                }
-
-                // show the dialog
-                $('#user-details-dialog').modal('show');
-            }
         }
     };
 }());
\ No newline at end of file


[13/36] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: 54e922c8f5740cab723d16b66d12b2a753c242ed
Parents: ec7f7e7 ed8f771
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 2 15:52:26 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 2 15:52:26 2015 -0500

----------------------------------------------------------------------
 nifi-nar-maven-plugin/pom.xml                   |   2 +-
 nifi/nifi-api/pom.xml                           |   4 +-
 nifi/nifi-assembly/pom.xml                      |   6 +-
 nifi/nifi-bootstrap/pom.xml                     |   2 +-
 .../nifi-data-provenance-utils/pom.xml          |   4 +-
 .../nifi-expression-language/pom.xml            |   4 +-
 .../nifi-commons/nifi-flowfile-packager/pom.xml |   4 +-
 nifi/nifi-commons/nifi-logging-utils/pom.xml    |   4 +-
 .../nifi-processor-utilities/pom.xml            |   4 +-
 nifi/nifi-commons/nifi-properties/pom.xml       |   4 +-
 nifi/nifi-commons/nifi-security-utils/pom.xml   |   4 +-
 nifi/nifi-commons/nifi-socket-utils/pom.xml     |   4 +-
 nifi/nifi-commons/nifi-utils/pom.xml            |   4 +-
 nifi/nifi-commons/nifi-web-utils/pom.xml        |   4 +-
 nifi/nifi-commons/nifi-write-ahead-log/pom.xml  |   4 +-
 nifi/nifi-commons/pom.xml                       |   5 +-
 nifi/nifi-docs/pom.xml                          |   6 +-
 .../nifi-processor-bundle-archetype/pom.xml     |   2 +-
 nifi/nifi-maven-archetypes/pom.xml              |   2 +-
 nifi/nifi-mock/pom.xml                          |   4 +-
 .../nifi-framework-nar/pom.xml                  |   4 +-
 .../nifi-framework/nifi-administration/pom.xml  |   4 +-
 .../nifi-framework/nifi-client-dto/pom.xml      |   2 +-
 .../nifi-cluster-authorization-provider/pom.xml |   2 +-
 .../nifi-cluster-protocol/pom.xml               |   2 +-
 .../nifi-framework/nifi-cluster-web/pom.xml     |   2 +-
 .../nifi-framework/nifi-cluster/pom.xml         |   2 +-
 .../nifi-file-authorization-provider/pom.xml    |   4 +-
 .../nifi-framework-core-api/pom.xml             |   4 +-
 .../nifi-framework/nifi-framework-core/pom.xml  |   2 +-
 .../nifi-framework/nifi-nar-utils/pom.xml       |   2 +-
 .../nifi-framework/nifi-resources/pom.xml       |   2 +-
 .../src/main/resources/conf/logback.xml         |   4 +-
 .../nifi-framework/nifi-runtime/pom.xml         |   2 +-
 .../nifi-framework/nifi-security/pom.xml        |   4 +-
 .../nifi-framework/nifi-site-to-site/pom.xml    |   4 +-
 .../nifi-framework/nifi-user-actions/pom.xml    |   4 +-
 .../nifi-web/nifi-custom-ui-utilities/pom.xml   |   2 +-
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml  |   2 +-
 .../nifi-web/nifi-web-api/pom.xml               |   4 +-
 .../nifi/web/controller/ControllerFacade.java   |  15 +-
 .../nifi-web/nifi-web-docs/pom.xml              |   2 +-
 .../nifi-web/nifi-web-error/pom.xml             |   2 +-
 .../nifi-web-optimistic-locking/pom.xml         |   2 +-
 .../nifi-web/nifi-web-security/pom.xml          |   2 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |   2 +-
 .../src/main/webapp/css/provenance.css          |   5 +
 .../webapp/js/nf/cluster/nf-cluster-table.js    | 319 ++++++-----
 .../webapp/js/nf/counters/nf-counters-table.js  |  63 ++-
 .../webapp/js/nf/history/nf-history-table.js    | 147 +++---
 .../js/nf/provenance/nf-provenance-table.js     | 103 ++--
 .../webapp/js/nf/summary/nf-summary-table.js    | 523 +++++++------------
 .../src/main/webapp/js/nf/summary/nf-summary.js |   6 -
 .../js/nf/templates/nf-templates-table.js       |  73 ++-
 .../main/webapp/js/nf/users/nf-users-table.js   | 440 ++++++++--------
 .../nifi-framework/nifi-web/pom.xml             |  10 +-
 .../nifi-framework/pom.xml                      |   4 +-
 .../nifi-framework-bundle/pom.xml               |  36 +-
 .../nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml  |   4 +-
 .../nifi-hdfs-processors/pom.xml                |   2 +-
 .../nifi-nar-bundles/nifi-hadoop-bundle/pom.xml |   6 +-
 .../nifi-hadoop-libraries-nar/pom.xml           |   4 +-
 .../nifi-hadoop-libraries-bundle/pom.xml        |   4 +-
 nifi/nifi-nar-bundles/nifi-jetty-bundle/pom.xml |   4 +-
 .../nifi-kafka-bundle/nifi-kafka-nar/pom.xml    |   4 +-
 .../nifi-kafka-processors/pom.xml               |   2 +-
 nifi/nifi-nar-bundles/nifi-kafka-bundle/pom.xml |   4 +-
 .../pom.xml                                     |   2 +-
 .../nifi-provenance-repository-nar/pom.xml      |   4 +-
 .../nifi-volatile-provenance-repository/pom.xml |   2 +-
 .../nifi-provenance-repository-bundle/pom.xml   |   8 +-
 .../nifi-standard-nar/pom.xml                   |   4 +-
 .../nifi-standard-prioritizers/pom.xml          |   2 +-
 .../nifi-standard-processors/pom.xml            |   2 +-
 .../nifi-standard-reporting-tasks/pom.xml       |   2 +-
 .../nifi-standard-bundle/pom.xml                |  10 +-
 .../pom.xml                                     |   2 +-
 .../pom.xml                                     |   2 +-
 .../nifi-distributed-cache-protocol/pom.xml     |   2 +-
 .../nifi-distributed-cache-server/pom.xml       |   2 +-
 .../nifi-distributed-cache-services-nar/pom.xml |   4 +-
 .../pom.xml                                     |   4 +-
 .../nifi-load-distribution-service-api/pom.xml  |   4 +-
 .../nifi-ssl-context-nar/pom.xml                |   4 +-
 .../nifi-ssl-context-service/pom.xml            |   2 +-
 .../nifi-ssl-context-bundle/pom.xml             |   4 +-
 .../nifi-ssl-context-service-api/pom.xml        |   2 +-
 .../nifi-standard-services-api-nar/pom.xml      |   4 +-
 .../nifi-standard-services/pom.xml              |   4 +-
 .../nifi-update-attribute-model/pom.xml         |   2 +-
 .../nifi-update-attribute-nar/pom.xml           |   4 +-
 .../nifi-update-attribute-processor/pom.xml     |   2 +-
 .../nifi-update-attribute-ui/pom.xml            |   2 +-
 .../update/attributes/api/RuleResource.java     |  16 +-
 .../nifi-update-attribute-bundle/pom.xml        |  10 +-
 nifi/nifi-nar-bundles/pom.xml                   |  31 +-
 nifi/pom.xml                                    |  64 +--
 97 files changed, 990 insertions(+), 1140 deletions(-)
----------------------------------------------------------------------



[07/36] incubator-nifi git commit: NIFI-297: - Fixed name of the nifi user log.

Posted by mc...@apache.org.
NIFI-297:
- Fixed name of the nifi user log.

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

Branch: refs/heads/NIFI-250
Commit: 94b39036dc10b025c321cf983dace0a3b9f45196
Parents: b4d07e2
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 13:16:34 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 13:16:34 2015 -0500

----------------------------------------------------------------------
 .../nifi-resources/src/main/resources/conf/logback.xml           | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94b39036/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
index 5f4e933..42d3353 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
@@ -42,7 +42,7 @@
     </appender>
     
     <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
-        <file>logs/${project.artifactId}-user.log</file>
+        <file>logs/nifi-user.log</file>
         <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
             <!--
               For daily rollover, use 'user_%d.log'.
@@ -50,7 +50,7 @@
               To GZIP rolled files, replace '.log' with '.log.gz'.
               To ZIP rolled files, replace '.log' with '.log.zip'.
             -->
-            <fileNamePattern>./logs/${project.artifactId}-user_%d.log</fileNamePattern>
+            <fileNamePattern>./logs/nifi-user_%d.log</fileNamePattern>
             <!-- keep 30 log files worth of history -->
             <maxHistory>30</maxHistory>
         </rollingPolicy>


[20/36] incubator-nifi git commit: NIFI-291 completed incorporation of Purtell feedback

Posted by mc...@apache.org.
NIFI-291 completed incorporation of Purtell feedback


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

Branch: refs/heads/NIFI-250
Commit: a738be8f27e86fab7cf8c7bd23285fb9cb016cad
Parents: d926aca
Author: joewitt <jo...@apache.org>
Authored: Tue Feb 3 09:43:19 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Tue Feb 3 09:43:19 2015 -0500

----------------------------------------------------------------------
 nifi-nar-maven-plugin/pom.xml | 2 +-
 nifi/pom.xml                  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a738be8f/nifi-nar-maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-maven-plugin/pom.xml b/nifi-nar-maven-plugin/pom.xml
index 6f46397..7ba2d7c 100644
--- a/nifi-nar-maven-plugin/pom.xml
+++ b/nifi-nar-maven-plugin/pom.xml
@@ -27,7 +27,7 @@
     <version>1.0.1-incubating-SNAPSHOT</version>
     <packaging>maven-plugin</packaging>
     <description>Apache NiFi Nar Plugin. It is currently a part of the Apache Incubator.</description>
-    <url>http://nifi.incubator.apache.org/maven-site/</url>
+    <url>http://nifi.incubator.apache.org</url>
     <organization>
         <name>Apache NiFi (incubating)</name>
         <url>http://nifi.incubator.apache.org/</url>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a738be8f/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/pom.xml b/nifi/pom.xml
index fb73f9c..8eda682 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -26,7 +26,7 @@
     <version>0.0.2-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
     <description>Apache NiFi is a dataflow system based on the concepts of flow-based programming. It is currently a part of the Apache Incubator.</description>
-    <url>http://nifi.incubator.apache.org/maven-site/</url>
+    <url>http://nifi.incubator.apache.org</url>
     <organization>
         <name>Apache NiFi (incubating)</name>
         <url>http://nifi.incubator.apache.org/</url>


[03/36] incubator-nifi git commit: NIFI-305: Cleaning up for extensibility; final methods

Posted by mc...@apache.org.
NIFI-305: Cleaning up for extensibility; final methods


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

Branch: refs/heads/NIFI-250
Commit: 615794e77fafc79ec6027484e584f76695a89573
Parents: ad40903
Author: gresockj <jg...@gmail.com>
Authored: Thu Jan 29 17:11:56 2015 -0500
Committer: gresockj <jg...@gmail.com>
Committed: Thu Jan 29 17:11:56 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/BinFiles.java      | 31 +++++++++++++++-----
 1 file changed, 24 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/615794e7/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
index b838d51..0a65c59 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
@@ -106,7 +106,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
     private final Queue<Bin> readyBins = new LinkedBlockingQueue<>();
 
     @Override
-    protected void init(final ProcessorInitializationContext context) {
+    protected final void init(final ProcessorInitializationContext context) {
 
     	final Set<Relationship> relationships = new HashSet<>();
         relationships.add(REL_ORIGINAL);
@@ -133,7 +133,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
     }
 
     @OnStopped
-    public void resetState() {
+    public final void resetState() {
         binManager.purge();
 
         Bin bin;
@@ -145,12 +145,12 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
     }
 
     @Override
-    public Set<Relationship> getRelationships() {
+    public final Set<Relationship> getRelationships() {
         return relationships;
     }
 
     @Override
-    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+    protected final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         return descriptors;
     }
 
@@ -222,8 +222,20 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
 			List<FlowFileSessionWrapper> binContents, ProcessContext context,
 			ProcessSession session, ProcessorLog logger) throws Exception;
 
+    /**
+	 * Allows additional custom validation to be done. This will be called from
+	 * the parent's customValidation method.
+	 * 
+	 * @param context
+	 *            The context
+	 * @return Validation results indicating problems
+	 */
+    protected Collection<ValidationResult> additionalCustomValidation(final ValidationContext context) {
+    	return new ArrayList<ValidationResult>();
+    }
+	
     @Override
-    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
         int binsAdded = binFlowFiles(context, sessionFactory);
         getLogger().debug("Binned {} FlowFiles", new Object[] {binsAdded});
         
@@ -336,7 +348,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
     }
 
     @OnScheduled
-    public void onScheduled(final ProcessContext context) throws IOException {
+    public final void onScheduled(final ProcessContext context) throws IOException {
         binManager.setMinimumSize(context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue());
 
         if (context.getProperty(MAX_BIN_AGE).isSet() ) {
@@ -363,7 +375,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
     }
     
 	@Override
-    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+    protected final Collection<ValidationResult> customValidate(final ValidationContext context) {
         final List<ValidationResult> problems = new ArrayList<>(super.customValidate(context));
 
         final long minBytes = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
@@ -382,6 +394,11 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
                 problems.add(new ValidationResult.Builder().subject(MIN_ENTRIES.getName()).input(context.getProperty(MIN_ENTRIES).getValue()).valid(false).explanation("Min Entries must be less than or equal to Max Entries").build());
             }
         }
+        
+        Collection<ValidationResult> otherProblems = this.additionalCustomValidation(context);
+        if (otherProblems != null) {
+        	problems.addAll(otherProblems);
+        }
 
         return problems;
     }


[04/36] incubator-nifi git commit: NIFI-269: fix OSX build

Posted by mc...@apache.org.
NIFI-269: fix OSX build


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

Branch: refs/heads/NIFI-250
Commit: 51b34a060ca9b59509360d5eb57a8ac325023142
Parents: 037f36d
Author: David Moravek <da...@gmail.com>
Authored: Mon Feb 2 09:48:18 2015 +0100
Committer: David Moravek <da...@gmail.com>
Committed: Mon Feb 2 09:51:11 2015 +0100

----------------------------------------------------------------------
 nifi/nifi-assembly/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/51b34a06/nifi/nifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml
index bc39b79..a3bcadc 100644
--- a/nifi/nifi-assembly/pom.xml
+++ b/nifi/nifi-assembly/pom.xml
@@ -43,6 +43,7 @@
                             <descriptors>
                                 <descriptor>src/main/assembly/dependencies.xml</descriptor>
                             </descriptors>
+                            <tarLongFileMode>posix</tarLongFileMode>
                         </configuration>
                     </execution>
                 </executions>    


[17/36] incubator-nifi git commit: NIFI-269 removed declaration of gnu tar form from parent pom

Posted by mc...@apache.org.
NIFI-269 removed declaration of gnu tar form from parent pom


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

Branch: refs/heads/NIFI-250
Commit: 2c2d39c7a24dc77a84e1e095280239e0d56c8357
Parents: fbfebf6
Author: joewitt <jo...@apache.org>
Authored: Tue Feb 3 08:18:58 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Tue Feb 3 08:18:58 2015 -0500

----------------------------------------------------------------------
 nifi/pom.xml | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2c2d39c7/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 765ef1b..499ffd6 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -842,9 +842,6 @@
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-assembly-plugin</artifactId>
                     <version>2.5.2</version>
-                    <configuration>
-                        <tarLongFileMode>gnu</tarLongFileMode>
-                    </configuration>
                 </plugin>
                 <plugin>
                     <groupId>org.codehaus.mojo</groupId>


[14/36] incubator-nifi git commit: NIFI-162: - Initial commit of the updated website. Contains copies of current pages and links to existing documents.

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/markdown/release-guide.md
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/markdown/release-guide.md b/nifi-site/src/pages/markdown/release-guide.md
new file mode 100644
index 0000000..e7750c1
--- /dev/null
+++ b/nifi-site/src/pages/markdown/release-guide.md
@@ -0,0 +1,286 @@
+---
+title: Apache NiFi Release Guidelines
+---
+
+The purpose of this document is to capture and describe the steps involved in producing 
+an official release of Apache NiFi.  It is written specifically to someone acting in the
+capacity of a [Release Manager][release-manager] (RM).  
+
+## Background Material
+
+  - These documents are necessary for all committers to be familiar with
+    - [Apache License V2.0][apache-license]
+    - [Apache Legal License/Resolved][apache-legal-resolve]
+    - [Apache How-to Apply License][apache-license-apply]
+    - [Apache Incubator Branding Guidelines][incubator-branding-guidelines]
+
+  - These documents are necessary for someone acting as the RM
+    - [Apache Encryption Software / ECCN Info][apache-encryption]
+    - [Apache Release Policy][apache-release-policy]
+    - [Apache Release Guide][apache-release-guide]
+    - [Apache Incubator Release Guide][apache-incubator-release-guide]
+    - [another Apache Incubator Release Guide][another-apache-incubator-release-guide]
+    - [Apache Incubator Policy][apache-incubator-policy]
+
+  - These documents are helpful for general environmental setup to perform releases
+    - [Apache PGP Info][apache-pgp]
+    - [Apache Release Signing][apache-release-signing]
+    - [Apache Guide to publish Maven Artifacts][apache-guide-publish-maven]
+
+## The objective
+
+Our aim is to produce and official Apache release.  
+The following is a list of the sorts of things that will be validated and are the basics to check
+when evaluating a release for a vote.
+
+## What to validate and how to Validate a release
+
+There are two lists here: one of specific incubator requirements, and another of general Apache requirements.
+
+### Incubator:
+
+  - Do the resulting artifacts have 'incubating' in the name?
+  - Is there a DISCLAIMER file in the source root that meets the requirements of the Incubator branding guidelines?
+
+### General Apache Release Requirements:
+
+  - Are LICENSE and NOTICE file present in the source root and complete?
+    - Specifically look in the *-sources.zip artifact and ensure these items are present at the root of the archive.
+  - Evaluate the sources and dependencies.  Does the overall LICENSE and NOTICE appear correct?  Do all licenses fit within the ASF approved licenses?
+    - Here is an example path to a sources artifact:  
+      - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi-nar-maven-plugin/0.0.1-incubating/nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip`
+  - Is there a README available that explains how to build the application and to execute it?
+    - Look in the *-sources.zip artifact root for the readme.
+  - Are the signatures and hashes correct for the source release?
+    - Validate the hashes of the sources artifact do in fact match:
+      - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi-nar-maven-plugin/0.0.1-incubating/nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip.md5`
+      - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi-nar-maven-plugin/0.0.1-incubating/nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip.sha1`
+    - Validate the signatures of the sources artifact and of each of the hashes.  Here are example paths:
+      - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi-nar-maven-plugin/0.0.1-incubating/nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip.asc`
+      - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi-nar-maven-plugin/0.0.1-incubating/nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip.asc.md5`
+      - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi-nar-maven-plugin/0.0.1-incubating/nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip.asc.sha1`
+      - Need a quick reminder on how to [verify a signature](http://www.apache.org/dev/release-signing.html#verifying-signature)?
+  - Do all sources have necessary headers?
+    - Unzip the sources file into a directory and execute `mvn install -Pcheck-licenses`
+  - Are there no unexpected binary files in the release?
+    - The only thing we'd expect would be potentially test resources files.
+  - Does the app (if appropriate) execute and function as expected?
+  
+## The flow of a release (an outline)
+  - The community is contributing to a series of JIRA tickets assigned to the next release
+  - The number of tickets open/remaining for that next release approaches zero
+  - A member of the community suggests a release and initiates a discussion
+  - Someone volunteers to be an RM for the release (can be a committer but apache guides indicate preference is a PPMC member)
+  - A release candidate is put together and a vote sent to the team.
+  - If the team rejects the vote the issues noted are resolved and another RC is generated
+  - Once a vote is accepted within the NiFi PPMC for a release candidate then the vote is sent to the IPMC
+  - If the IPMC rejects the vote then the issues are resolved and a new RC prepared and voted upon within the PPMC
+  - If the IPMC accepts the vote then the release is 'releasable' and can be placed into the appropriate 'dist' location, maven artifacts released from staging.
+  
+## The mechanics of the release
+
+### Prepare your environment
+  
+Follow the steps outlined in the [Quickstart Guide][quickstart-guide]
+        
+    At this point you're on the latest 'develop' branch and are able to build the entire application
+
+Create a JIRA ticket for the release tasks and use that ticket number for the commit messages.  For example we'll consider NIFI-270 as our ticket.  Also
+have in mind the release version you are planning for.  For example we'll consider '0.0.1-incubating'.
+
+Create the next version in JIRA if necessary so develop work can continue towards that release.
+
+Create new branch off develop named after the JIRA ticket or just use the develop branch itself.  Here we'll use a branch off of develop with
+`git checkout -b NIFI-270`
+
+Change directory into that of the project you wish to release.  For example either `cd nifi` or `cd nifi-nar-maven-plugin`
+
+Verify that Maven has sufficient heap space to perform the build tasks.  Some plugins and parts of the build 
+consumes a surprisingly large amount of space.  These settings have been shown to 
+work `MAVEN_OPTS="-Xms1024m -Xmx3076m -XX:MaxPermSize=256m"`
+
+Ensure your settings.xml has been updated as shown below.  There are other ways to ensure your PGP key is available for signing as well
+  
+>          ...
+>          <profile>
+>             <id>signed_release</id>
+>             <properties>
+>                 <mavenExecutorId>forked-path</mavenExecutorId>
+>                 <gpg.keyname>YOUR GPG KEY ID HERE</gpg.keyname>
+>                 <gpg.passphrase>YOUR GPG PASSPHRASE HERE</gpg.passphrase>
+>             </properties>
+>         </profile>
+>         ...
+>         <servers>
+>            <server>
+>                <id>repository.apache.org</id>
+>                <username>YOUR USER NAME HERE</username>
+>                <password>YOUR MAVEN ENCRYPTED PASSWORD HERE</password>
+>            </server>
+>         </servers>
+>         ...
+
+Ensure the the full application build and tests all work by executing
+`mvn -T 2.5C clean install` for a parallel build.  Once that completes you can
+startup and test the application by `cd assembly/target` then run `bin/nifi.sh start` in the nifi build.
+The application should be up and running in a few seconds at `http://localhost:8080/nifi`
+
+Evaluate and ensure the appropriate license headers are present on all source files.  Ensure LICENSE and NOTICE files are complete and accurate.  
+Developers should always be keeping these up to date as they go along adding source and modifying dependencies to keep this burden manageable.  
+This command `mvn install -Pcheck-licenses` should be run as well to help validate.  If that doesn't complete cleanly it must be addressed.
+
+Now its time to have maven prepare the release so execute `mvn release:prepare -Psigned_release -DscmCommentPrefix="NIFI-270 " -Darguments="-DskipTests"`.
+Maven will ask:
+
+`What is the release version for "Apache NiFi NAR Plugin"? (org.apache.nifi:nifi-nar-maven-plugin) 0.0.1-incubating: :`
+
+Just hit enter to accept the default.
+
+Maven will then ask:
+
+`What is SCM release tag or label for "Apache NiFi NAR Plugin"? (org.apache.nifi:nifi-nar-maven-plugin) nifi-nar-maven-plugin-0.0.1-incubating: : `
+
+Enter `nifi-nar-maven-plugin-0.0.1-incubating-RC1` or whatever the appropriate release candidate (RC) number is.
+Maven will then ask:
+
+`What is the new development version for "Apache NiFi NAR Plugin"? (org.apache.nifi:nifi-nifi-nar-maven-plugin) 0.0.2-incubating-SNAPSHOT: :`
+
+Just hit enter to accept the default.
+
+Now that preparation went perfectly it is time to perform the release and deploy artifacts to staging.  To do that execute
+
+`mvn release:perform -Psigned_release -DscmCommentPrefix="NIFI-270 " -Darguments="-DskipTests"`
+
+That will complete successfully and this means the artifacts have been released to the Apache Nexus staging repository.  You will see something like
+
+`    [INFO]  * Closing staging repository with ID "orgapachenifi-1011".`
+
+So if you browse to `https://repository.apache.org/#stagingRepositories` login with your Apache committer credentials and you should see `orgapachenifi-1011`.  If you click on that you can inspect the various staged artifacts.
+
+Validate that all the various aspects of the staged artifacts appear correct
+
+  - Download the sources.  Do they compile cleanly?  If the result is a build does it execute?
+  - Validate the hashes match.
+  - Validate that the sources contain no unexpected binaries.
+  - Validate the signature for the build and hashes.
+  - Validate the LICENSE/NOTICE/DISCLAIMER/Headers.  
+  - Validate that the README is present and provides sufficient information to build and if necessary execute.
+  
+If all looks good then push the branch to origin `git push origin NIFI-270`
+
+If anything isn't correct about the staged artifacts you can drop the staged repo from repository.apache.org and delete the
+local tag in git.  If you also delete the local branch and clear your local maven repository under org/apache/nifi then it is
+as if the release never happened.  Before doing that though try to figure out what went wrong.  So as described here you see
+that you can pretty easily test the release process until you get it right.  The `mvn versions:set ` and `mvn versions:commit `
+commands can come in handy to help do this so you can set versions to something clearly release test related.
+
+Now it's time to initiate a vote within the PPMC.  Send the vote request to `dev@nifi.incubator.apache.org`
+with a subject of `[VOTE] Release Apache NiFi nifi-nar-maven-plugin-0.0.1-incubating`. The following template can be used:
+ 
+>     Hello
+>     I am pleased to be calling this vote for the source release of Apache NiFi
+>     nifi-nar-maven-plugin-0.0.1-incubating.
+>     
+>     The source zip, including signatures, digests, etc. can be found at:
+>     https://repository.apache.org/content/repositories/orgapachenifi-1011
+>     
+>     The Git tag is nifi-nar-maven-plugin-0.0.1-incubating-RC1
+>     The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
+>     https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
+>     
+>     Checksums of nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip:
+>     MD5: 5a580756a17b0573efa3070c70585698
+>     SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
+>     
+>     Release artifacts are signed with the following key:
+>     https://people.apache.org/keys/committer/joewitt.asc
+>     
+>     KEYS file available here:
+>     https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
+>     
+>     8 issues were closed/resolved for this release:
+>     https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
+>     
+>     The vote will be open for 72 hours. 
+>     Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test.  The please vote:
+>     
+>     [ ] +1 Release this package as nifi-nar-maven-plugin-0.0.1-incubating
+>     [ ] +0 no opinion
+>     [ ] -1 Do not release this package because because...
+
+A release vote is majority rule.  So wait 72 hours and see if there are at least 3 binding +1 votes and no more negative votes than positive.
+If so forward the vote to the IPMC.  Send the vote request to `general@incubator.apache.org` with a subject of
+`[VOTE] Release Apache NiFi nifi-nar-maven-plugin-0.0.1-incubating`.  The following template can be used:
+
+>     Hello
+>     
+>     The Apache NiFi PPMC has voted to release Apache NiFi nar-maven-plugin-0.0.1-incubating.
+>     The vote was based on the release candidate and thread described below.
+>     We now request the IPMC to vote on this release.
+>     
+>     Here is the PPMC voting result:
+>     X +1 (binding)
+>     Y -1 (binding)
+>     
+>     Here is the PPMC vote thread: [URL TO PPMC Vote Thread]
+>     
+>     The source zip, including signatures, digests, etc. can be found at:
+>     https://repository.apache.org/content/repositories/orgapachenifi-1011
+>     
+>     The Git tag is nar-maven-plugin-0.0.1-incubating-RC1
+>     The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
+>     https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
+>     
+>     Checksums of nar-maven-plugin-0.0.1-incubating-source-release.zip:
+>     MD5: 5a580756a17b0573efa3070c70585698
+>     SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
+>     
+>     Release artifacts are signed with the following key:
+>     https://people.apache.org/keys/committer/joewitt.asc
+>     
+>     KEYS file available here:
+>     https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
+>     
+>     8 issues were closed/resolved for this release:
+>     https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
+>     
+>     The vote will be open for 72 hours. 
+>     Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test.  The please vote:
+>     
+>     [ ] +1 Release this package as nar-maven-plugin-0.0.1-incubating
+>     [ ] +0 no opinion
+>     [ ] -1 Do not release this package because because...
+
+Wait 72 hours.  If the vote passes then send a vote result email.  Send the email to `general@incubator.apache.org, dev@nifi.incubator.apache.org`
+with a subject of `[RESULT][VOTE] Release Apache NiFi nar-maven-plugin-0.0.1-incubating`.  Use a template such as:
+
+>     Hello
+>     
+>     The release passes with
+>     
+>     X +1 (binding) votes
+>     Y -1 (binding) votes
+>     
+>     Thanks to all who helped make this release possible.
+>     
+>     Here is the IPMC vote thread: [INSERT URL OF IPMC Vote Thread]
+
+Now all the voting is done and the release is good to go.  In repository.apache.org go to the staging repository
+and select `release`.  Then publish the source, hashes, and signatures to `https://dist.apache.org/repos/dist/release/incubator/nifi/`
+Then merge the release git tag to develop and to master.
+
+[quickstart-guide]: http://nifi.incubator.apache.org/development/quickstart.html
+[release-manager]: http://www.apache.org/dev/release-publishing.html#release_manager
+[apache-license]: http://apache.org/licenses/LICENSE-2.0
+[apache-license-apply]: http://www.apache.org/dev/apply-license.html
+[apache-legal-resolve]: http://www.apache.org/legal/resolved.html
+[apache-encryption]: http://www.apache.org/licenses/exports/
+[apache-release-policy]: http://www.apache.org/dev/release.html
+[apache-release-guide]: http://www.apache.org/dev/release-publishing
+[apache-incubator-release-guide]: http://incubator.apache.org/guides/releasemanagement.html
+[another-apache-incubator-release-guide]: http://incubator.apache.org/guides/release.html
+[apache-incubator-policy]: http://incubator.apache.org/incubation/Incubation_Policy.html
+[incubator-branding-guidelines]: http://incubator.apache.org/guides/branding.html
+[apache-pgp]: http://www.apache.org/dev/openpgp.html
+[apache-release-signing]: http://www.apache.org/dev/release-signing.html
+[apache-guide-publish-maven]: http://www.apache.org/dev/publishing-maven-artifacts.html
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/scss/_settings.scss
----------------------------------------------------------------------
diff --git a/nifi-site/src/scss/_settings.scss b/nifi-site/src/scss/_settings.scss
new file mode 100644
index 0000000..ff2d464
--- /dev/null
+++ b/nifi-site/src/scss/_settings.scss
@@ -0,0 +1,1458 @@
+// Foundation by ZURB
+// foundation.zurb.com
+// Licensed under MIT Open Source
+
+//
+
+// Table of Contents
+// Foundation Settings
+//
+// a. Base
+// b. Grid
+// c. Global
+// d. Media Query Ranges
+// e. Typography
+// 01. Accordion
+// 02. Alert Boxes
+// 03. Block Grid
+// 04. Breadcrumbs
+// 05. Buttons
+// 06. Button Groups
+// 07. Clearing
+// 08. Dropdown
+// 09. Dropdown Buttons
+// 10. Flex Video
+// 11. Forms
+// 12. Icon Bar
+// 13. Inline Lists
+// 14. Joyride
+// 15. Keystrokes
+// 16. Labels
+// 17. Magellan
+// 18. Off-canvas
+// 19. Orbit
+// 20. Pagination
+// 21. Panels
+// 22. Pricing Tables
+// 23. Progress Bar
+// 24. Range Slider
+// 25. Reveal
+// 26. Side Nav
+// 27. Split Buttons
+// 28. Sub Nav
+// 29. Switch
+// 30. Tables
+// 31. Tabs
+// 32. Thumbnails
+// 33. Tooltips
+// 34. Top Bar
+// 36. Visibility Classes
+
+// a. Base
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// This is the default html and body font-size for the base rem value.
+// $rem-base: 16px;
+
+// Allows the use of rem-calc() or lower-bound() in your settings
+@import "foundation/functions";
+
+// The default font-size is set to 100% of the browser style sheet (usually 16px)
+// for compatibility with browser-based text zoom or user-set defaults.
+
+// Since the typical default browser font-size is 16px, that makes the calculation for grid size.
+// If you want your base font-size to be different and not have it affect the grid breakpoints,
+// set $rem-base to $base-font-size and make sure $base-font-size is a px value.
+// $base-font-size: 100%;
+
+// The $base-font-size is 100% while $base-line-height is 150%
+// $base-line-height: 150%;
+
+// We use this to control whether or not CSS classes come through in the gem files.
+$include-html-classes: true;
+// $include-print-styles: true;
+$include-html-global-classes: $include-html-classes;
+
+// b. Grid
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-grid-classes: $include-html-classes;
+// $include-xl-html-grid-classes: false;
+
+// $row-width: rem-calc(1000);
+// $total-columns: 12;
+// $column-gutter: rem-calc(30);
+
+// c. Global
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// We use these to define default font stacks
+// $font-family-sans-serif: "Helvetica Neue", Helvetica, Roboto, Arial, sans-serif;
+// $font-family-serif: Georgia, Cambria, "Times New Roman", Times, serif;
+// $font-family-monospace: Consolas, "Liberation Mono", Courier, monospace;
+
+// We use these to define default font weights
+// $font-weight-normal: normal !default;
+$font-weight-bold: bold !default;
+
+// $white       : #FFFFFF;
+// $ghost       : #FAFAFA;
+// $snow        : #F9F9F9;
+// $vapor       : #F6F6F6;
+// $white-smoke : #F5F5F5;
+// $silver      : #EFEFEF;
+// $smoke       : #EEEEEE;
+// $gainsboro   : #DDDDDD;
+// $iron        : #CCCCCC;
+// $base        : #AAAAAA;
+// $aluminum    : #999999;
+// $jumbo       : #888888;
+// $monsoon     : #777777;
+// $steel       : #666666;
+// $charcoal    : #555555;
+// $tuatara     : #444444;
+// $oil         : #333333;
+// $jet         : #222222;
+// $black       : #000000;
+
+// We use these as default colors throughout
+// $primary-color: #008CBA;
+// $secondary-color: #e7e7e7;
+// $alert-color: #f04124;
+// $success-color: #43AC6A;
+// $warning-color: #f08a24;
+// $info-color: #a0d3e8;
+
+// We use these to control various global styles
+// $body-bg: $white;
+// $body-font-color: $jet;
+// $body-font-family: $font-family-sans-serif;
+// $body-font-weight: $font-weight-normal;
+// $body-font-style: normal;
+
+// We use this to control font-smoothing
+// $font-smoothing: antialiased;
+
+// We use these to control text direction settings
+// $text-direction: ltr;
+// $opposite-direction: right;
+// $default-float: left;
+// $last-child-float: $opposite-direction;
+
+// We use these to make sure border radius matches unless we want it different.
+// $global-radius: 3px;
+// $global-rounded: 1000px;
+
+// We use these to control inset shadow shiny edges and depressions.
+// $shiny-edge-size: 0 1px 0;
+// $shiny-edge-color: rgba($white, .5);
+// $shiny-edge-active-color: rgba($black, .2);
+
+// d. Media Query Ranges
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $small-range: (0em, 40em);
+// $medium-range: (40.063em, 64em);
+// $large-range: (64.063em, 90em);
+// $xlarge-range: (90.063em, 120em);
+// $xxlarge-range: (120.063em, 99999999em);
+
+// $screen: "only screen";
+
+// $landscape: "#{$screen} and (orientation: landscape)";
+// $portrait: "#{$screen} and (orientation: portrait)";
+
+// $small-up: $screen;
+// $small-only: "#{$screen} and (max-width: #{upper-bound($small-range)})";
+
+// $medium-up: "#{$screen} and (min-width:#{lower-bound($medium-range)})";
+// $medium-only: "#{$screen} and (min-width:#{lower-bound($medium-range)}) and (max-width:#{upper-bound($medium-range)})";
+
+// $large-up: "#{$screen} and (min-width:#{lower-bound($large-range)})";
+// $large-only: "#{$screen} and (min-width:#{lower-bound($large-range)}) and (max-width:#{upper-bound($large-range)})";
+
+// $xlarge-up: "#{$screen} and (min-width:#{lower-bound($xlarge-range)})";
+// $xlarge-only: "#{$screen} and (min-width:#{lower-bound($xlarge-range)}) and (max-width:#{upper-bound($xlarge-range)})";
+
+// $xxlarge-up: "#{$screen} and (min-width:#{lower-bound($xxlarge-range)})";
+// $xxlarge-only: "#{$screen} and (min-width:#{lower-bound($xxlarge-range)}) and (max-width:#{upper-bound($xxlarge-range)})";
+
+// Legacy
+// $small: $medium-up;
+// $medium: $medium-up;
+// $large: $large-up;
+
+// We use this as cursors values for enabling the option of having custom cursors in the whole site's stylesheet
+// $cursor-crosshair-value: crosshair;
+// $cursor-default-value: default;
+// $cursor-pointer-value: pointer;
+// $cursor-help-value: help;
+// $cursor-text-value: text;
+
+// e. Typography
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-type-classes: $include-html-classes;
+
+// We use these to control header font styles
+// $header-font-family: $body-font-family;
+// $header-font-weight: $font-weight-normal;
+// $header-font-style: normal;
+// $header-font-color: $jet;
+// $header-line-height: 1.4;
+// $header-top-margin: .2rem;
+// $header-bottom-margin: .5rem;
+// $header-text-rendering: optimizeLegibility;
+
+// We use these to control header font sizes
+// $h1-font-size: rem-calc(44);
+// $h2-font-size: rem-calc(37);
+// $h3-font-size: rem-calc(27);
+// $h4-font-size: rem-calc(23);
+// $h5-font-size: rem-calc(18);
+// $h6-font-size: 1rem;
+
+// We use these to control header size reduction on small screens
+// $h1-font-reduction: rem-calc(10) !default;
+// $h2-font-reduction: rem-calc(10) !default;
+// $h3-font-reduction: rem-calc(5) !default;
+// $h4-font-reduction: rem-calc(5) !default;
+// $h5-font-reduction: 0 !default;
+// $h6-font-reduction: 0 !default;
+
+// These control how subheaders are styled.
+// $subheader-line-height: 1.4;
+// $subheader-font-color: scale-color($header-font-color, $lightness: 35%);
+// $subheader-font-weight: $font-weight-normal;
+// $subheader-top-margin: .2rem;
+// $subheader-bottom-margin: .5rem;
+
+// A general <small> styling
+// $small-font-size: 60%;
+// $small-font-color: scale-color($header-font-color, $lightness: 35%);
+
+// We use these to style paragraphs
+// $paragraph-font-family: inherit;
+// $paragraph-font-weight: $font-weight-normal;
+// $paragraph-font-size: 1rem;
+// $paragraph-line-height: 1.6;
+// $paragraph-margin-bottom: rem-calc(20);
+// $paragraph-aside-font-size: rem-calc(14);
+// $paragraph-aside-line-height: 1.35;
+// $paragraph-aside-font-style: italic;
+// $paragraph-text-rendering: optimizeLegibility;
+
+// We use these to style <code> tags
+// $code-color: $oil;
+// $code-font-family: $font-family-monospace;
+// $code-font-weight: $font-weight-normal;
+// $code-background-color: scale-color($secondary-color, $lightness: 70%);
+// $code-border-size: 1px;
+// $code-border-style: solid;
+// $code-border-color: scale-color($code-background-color, $lightness: -10%);
+// $code-padding: rem-calc(2) rem-calc(5) rem-calc(1);
+
+// We use these to style anchors
+// $anchor-text-decoration: none;
+// $anchor-text-decoration-hover: none;
+// $anchor-font-color: $primary-color;
+$anchor-font-color: #1e373f;
+// $anchor-font-color-hover: scale-color($primary-color, $lightness: -14%);
+$anchor-font-color: #264c58;
+
+// We use these to style the <hr> element
+// $hr-border-width: 1px;
+// $hr-border-style: solid;
+// $hr-border-color: $gainsboro;
+// $hr-margin: rem-calc(20);
+
+// We use these to style lists
+// $list-font-family: $paragraph-font-family;
+// $list-font-size: $paragraph-font-size;
+// $list-line-height: $paragraph-line-height;
+// $list-margin-bottom: $paragraph-margin-bottom;
+// $list-style-position: outside;
+// $list-side-margin: 1.1rem;
+$list-side-margin: 2.0rem;
+// $list-ordered-side-margin: 1.4rem;
+// $list-side-margin-no-bullet: 0;
+// $list-nested-margin: rem-calc(20);
+// $definition-list-header-weight: $font-weight-bold;
+// $definition-list-header-margin-bottom: .3rem;
+// $definition-list-margin-bottom: rem-calc(12);
+
+// We use these to style blockquotes
+// $blockquote-font-color: scale-color($header-font-color, $lightness: 35%);
+// $blockquote-padding: rem-calc(9 20 0 19);
+// $blockquote-border: 1px solid $gainsboro;
+// $blockquote-cite-font-size: rem-calc(13);
+// $blockquote-cite-font-color: scale-color($header-font-color, $lightness: 23%);
+// $blockquote-cite-link-color: $blockquote-cite-font-color;
+
+// Acronym styles
+// $acronym-underline: 1px dotted $gainsboro;
+
+// We use these to control padding and margin
+// $microformat-padding: rem-calc(10 12);
+// $microformat-margin: rem-calc(0 0 20 0);
+
+// We use these to control the border styles
+// $microformat-border-width: 1px;
+// $microformat-border-style: solid;
+// $microformat-border-color: $gainsboro;
+
+// We use these to control full name font styles
+// $microformat-fullname-font-weight: $font-weight-bold;
+// $microformat-fullname-font-size: rem-calc(15);
+
+// We use this to control the summary font styles
+// $microformat-summary-font-weight: $font-weight-bold;
+
+// We use this to control abbr padding
+// $microformat-abbr-padding: rem-calc(0 1);
+
+// We use this to control abbr font styles
+// $microformat-abbr-font-weight: $font-weight-bold;
+// $microformat-abbr-font-decoration: none;
+
+// 01. Accordion
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-accordion-classes: $include-html-classes;
+
+// $accordion-navigation-padding: rem-calc(16);
+// $accordion-navigation-bg-color: $silver;
+// $accordion-navigation-hover-bg-color: scale-color($accordion-navigation-bg-color, $lightness: -5%);
+// $accordion-navigation-active-bg-color: scale-color($accordion-navigation-bg-color, $lightness: -3%);
+// $accordion-navigation-font-color: $jet;
+// $accordion-navigation-font-size: rem-calc(16);
+// $accordion-navigation-font-family: $body-font-family;
+
+// $accordion-content-padding: ($column-gutter/2);
+// $accordion-content-active-bg-color: $white;
+
+// 02. Alert Boxes
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-alert-classes: $include-html-classes;
+
+// We use this to control alert padding.
+// $alert-padding-top: rem-calc(14);
+// $alert-padding-default-float: $alert-padding-top;
+// $alert-padding-opposite-direction: $alert-padding-top + rem-calc(10);
+// $alert-padding-bottom: $alert-padding-top;
+
+// We use these to control text style.
+// $alert-font-weight: $font-weight-normal;
+// $alert-font-size: rem-calc(13);
+// $alert-font-color: $white;
+// $alert-font-color-alt: scale-color($secondary-color, $lightness: -66%);
+
+// We use this for close hover effect.
+// $alert-function-factor: -14%;
+
+// We use these to control border styles.
+// $alert-border-style: solid;
+// $alert-border-width: 1px;
+// $alert-border-color: scale-color($primary-color, $lightness: $alert-function-factor);
+// $alert-bottom-margin: rem-calc(20);
+
+// We use these to style the close buttons
+// $alert-close-color: $oil;
+// $alert-close-top: 50%;
+// $alert-close-position: rem-calc(4);
+// $alert-close-font-size: rem-calc(22);
+// $alert-close-opacity: 0.3;
+// $alert-close-opacity-hover: 0.5;
+// $alert-close-padding: 9px 6px 4px;
+
+// We use this to control border radius
+// $alert-radius: $global-radius;
+
+// We use this to control transition effects
+// $alert-transition-speed: 300ms;
+// $alert-transition-ease: ease-out;
+
+// 03. Block Grid
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-block-grid-classes: $include-html-classes;
+// $include-xl-html-block-grid-classes: false;
+
+// We use this to control the maximum number of block grid elements per row
+// $block-grid-elements: 12;
+// $block-grid-default-spacing: rem-calc(20);
+// $align-block-grid-to-grid: false;
+
+// Enables media queries for block-grid classes. Set to false if writing semantic HTML.
+// $block-grid-media-queries: true;
+
+// 04. Breadcrumbs
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-nav-classes: $include-html-classes;
+
+// We use this to set the background color for the breadcrumb container.
+// $crumb-bg: scale-color($secondary-color, $lightness: 55%);
+
+// We use these to set the padding around the breadcrumbs.
+// $crumb-padding: rem-calc(9 14 9);
+// $crumb-side-padding: rem-calc(12);
+
+// We use these to control border styles.
+// $crumb-function-factor: -10%;
+// $crumb-border-size: 1px;
+// $crumb-border-style: solid;
+// $crumb-border-color: scale-color($crumb-bg, $lightness: $crumb-function-factor);
+// $crumb-radius: $global-radius;
+
+// We use these to set various text styles for breadcrumbs.
+// $crumb-font-size: rem-calc(11);
+// $crumb-font-color: $primary-color;
+// $crumb-font-color-current: $oil;
+// $crumb-font-color-unavailable: $aluminum;
+// $crumb-font-transform: uppercase;
+// $crumb-link-decor: underline;
+
+// We use these to control the slash between breadcrumbs
+// $crumb-slash-color: $base;
+// $crumb-slash: "/";
+
+// 05. Buttons
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-button-classes: $include-html-classes;
+
+// We use these to build padding for buttons.
+// $button-tny: rem-calc(10);
+// $button-sml: rem-calc(14);
+// $button-med: rem-calc(16);
+// $button-lrg: rem-calc(18);
+
+// We use this to control the display property.
+// $button-display: inline-block;
+// $button-margin-bottom: rem-calc(20);
+
+// We use these to control button text styles.
+// $button-font-family: $body-font-family;
+// $button-font-color: $white;
+// $button-font-color-alt: $oil;
+// $button-font-tny: rem-calc(11);
+// $button-font-sml: rem-calc(13);
+// $button-font-med: rem-calc(16);
+// $button-font-lrg: rem-calc(20);
+// $button-font-weight: $font-weight-normal;
+// $button-font-align: center;
+
+// We use these to control various hover effects.
+// $button-function-factor: -20%;
+
+// We use these to control button border and hover styles.
+// $button-border-width: 0px;
+// $button-border-style: solid;
+// $button-bg-color: $primary-color;
+// $button-bg-hover: scale-color($button-bg-color, $lightness: $button-function-factor);
+// $button-border-color: $button-bg-hover;
+// $secondary-button-bg-hover: scale-color($secondary-color, $lightness: $button-function-factor);
+// $secondary-button-border-color: $secondary-button-bg-hover;
+// $success-button-bg-hover: scale-color($success-color, $lightness: $button-function-factor);
+// $success-button-border-color: $success-button-bg-hover;
+// $alert-button-bg-hover: scale-color($alert-color, $lightness: $button-function-factor);
+// $alert-button-border-color: $alert-button-bg-hover;
+
+// We use this to set the default radius used throughout the core.
+// $button-radius: $global-radius;
+// $button-round: $global-rounded;
+
+// We use this to set default opacity and cursor for disabled buttons.
+// $button-disabled-opacity: 0.7;
+// $button-disabled-cursor: $cursor-default-value;
+
+// 06. Button Groups
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-button-classes: $include-html-classes;
+
+// Sets the margin for the right side by default, and the left margin if right-to-left direction is used
+// $button-bar-margin-opposite: rem-calc(10);
+// $button-group-border-width: 1px;
+
+// 07. Clearing
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-clearing-classes: $include-html-classes;
+
+// We use these to set the background colors for parts of Clearing.
+// $clearing-bg: $oil;
+// $clearing-caption-bg: $clearing-bg;
+// $clearing-carousel-bg: rgba(51,51,51,0.8);
+// $clearing-img-bg: $clearing-bg;
+
+// We use these to style the close button
+// $clearing-close-color: $iron;
+// $clearing-close-size: 30px;
+
+// We use these to style the arrows
+// $clearing-arrow-size: 12px;
+// $clearing-arrow-color: $clearing-close-color;
+
+// We use these to style captions
+// $clearing-caption-font-color: $iron;
+// $clearing-caption-font-size: 0.875em;
+// $clearing-caption-padding: 10px 30px 20px;
+
+// We use these to make the image and carousel height and style
+// $clearing-active-img-height: 85%;
+// $clearing-carousel-height: 120px;
+// $clearing-carousel-thumb-width: 120px;
+// $clearing-carousel-thumb-active-border: 1px solid rgb(255,255,255);
+
+// 08. Dropdown
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-dropdown-classes: $include-html-classes;
+
+// We use these to controls height and width styles.
+// $f-dropdown-max-width: 200px;
+// $f-dropdown-height: auto;
+// $f-dropdown-max-height: none;
+
+// Used for bottom position
+// $f-dropdown-margin-top: 2px;
+
+// Used for right position
+// $f-dropdown-margin-left: $f-dropdown-margin-top;
+
+// Used for left position
+// $f-dropdown-margin-right: $f-dropdown-margin-top;
+
+// Used for top position
+// $f-dropdown-margin-bottom: $f-dropdown-margin-top;
+
+// We use this to control the background color
+// $f-dropdown-bg: $white;
+
+// We use this to set the border styles for dropdowns.
+// $f-dropdown-border-style: solid;
+// $f-dropdown-border-width: 1px;
+// $f-dropdown-border-color: scale-color($white, $lightness: -20%);
+
+// We use these to style the triangle pip.
+// $f-dropdown-triangle-size: 6px;
+// $f-dropdown-triangle-color: $white;
+// $f-dropdown-triangle-side-offset: 10px;
+
+// We use these to control styles for the list elements.
+// $f-dropdown-list-style: none;
+// $f-dropdown-font-color: $charcoal;
+// $f-dropdown-font-size: rem-calc(14);
+// $f-dropdown-list-padding: rem-calc(5, 10);
+// $f-dropdown-line-height: rem-calc(18);
+// $f-dropdown-list-hover-bg: $smoke;
+// $dropdown-mobile-default-float: 0;
+
+// We use this to control the styles for when the dropdown has custom content.
+// $f-dropdown-content-padding: rem-calc(20);
+
+// Default radius for dropdown.
+// $f-dropdown-radius: $global-radius;
+
+
+// 09. Dropdown Buttons
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-button-classes: $include-html-classes;
+
+// We use these to set the color of the pip in dropdown buttons
+// $dropdown-button-pip-color: $white;
+// $dropdown-button-pip-color-alt: $oil;
+
+// $button-pip-tny: rem-calc(6);
+// $button-pip-sml: rem-calc(7);
+// $button-pip-med: rem-calc(9);
+// $button-pip-lrg: rem-calc(11);
+
+// We use these to style tiny dropdown buttons
+// $dropdown-button-padding-tny: $button-pip-tny * 7;
+// $dropdown-button-pip-size-tny: $button-pip-tny;
+// $dropdown-button-pip-opposite-tny: $button-pip-tny * 3;
+// $dropdown-button-pip-top-tny: (-$button-pip-tny / 2) + rem-calc(1);
+
+// We use these to style small dropdown buttons
+// $dropdown-button-padding-sml: $button-pip-sml * 7;
+// $dropdown-button-pip-size-sml: $button-pip-sml;
+// $dropdown-button-pip-opposite-sml: $button-pip-sml * 3;
+// $dropdown-button-pip-top-sml: (-$button-pip-sml / 2) + rem-calc(1);
+
+// We use these to style medium dropdown buttons
+// $dropdown-button-padding-med: $button-pip-med * 6 + rem-calc(3);
+// $dropdown-button-pip-size-med: $button-pip-med - rem-calc(3);
+// $dropdown-button-pip-opposite-med: $button-pip-med * 2.5;
+// $dropdown-button-pip-top-med: (-$button-pip-med / 2) + rem-calc(2);
+
+// We use these to style large dropdown buttons
+// $dropdown-button-padding-lrg: $button-pip-lrg * 5 + rem-calc(3);
+// $dropdown-button-pip-size-lrg: $button-pip-lrg - rem-calc(6);
+// $dropdown-button-pip-opposite-lrg: $button-pip-lrg * 2.5;
+// $dropdown-button-pip-top-lrg: (-$button-pip-lrg / 2) + rem-calc(3);
+
+// 10. Flex Video
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-media-classes: $include-html-classes;
+
+// We use these to control video container padding and margins
+// $flex-video-padding-top: rem-calc(25);
+// $flex-video-padding-bottom: 67.5%;
+// $flex-video-margin-bottom: rem-calc(16);
+
+// We use this to control widescreen bottom padding
+// $flex-video-widescreen-padding-bottom: 56.34%;
+
+// 11. Forms
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-form-classes: $include-html-classes;
+
+// We use this to set the base for lots of form spacing and positioning styles
+// $form-spacing: rem-calc(16);
+
+// We use these to style the labels in different ways
+// $form-label-pointer: pointer;
+// $form-label-font-size: rem-calc(14);
+// $form-label-font-weight: $font-weight-normal;
+// $form-label-line-height: 1.5;
+// $form-label-font-color: scale-color($black, $lightness: 30%);
+// $form-label-small-transform: capitalize;
+// $form-label-bottom-margin: 0;
+// $input-font-family: inherit;
+// $input-font-color: rgba(0,0,0,0.75);
+// $input-font-size: rem-calc(14);
+// $input-bg-color: $white;
+// $input-focus-bg-color: scale-color($white, $lightness: -2%);
+// $input-border-color: scale-color($white, $lightness: -20%);
+// $input-focus-border-color: scale-color($white, $lightness: -40%);
+// $input-border-style: solid;
+// $input-border-width: 1px;
+// $input-border-radius: $global-radius;
+// $input-disabled-bg: $gainsboro;
+// $input-disabled-cursor: $cursor-default-value;
+// $input-box-shadow: inset 0 1px 2px rgba(0,0,0,0.1);
+
+// We use these to style the fieldset border and spacing.
+// $fieldset-border-style: solid;
+// $fieldset-border-width: 1px;
+// $fieldset-border-color: $gainsboro;
+// $fieldset-padding: rem-calc(20);
+// $fieldset-margin: rem-calc(18 0);
+
+// We use these to style the legends when you use them
+// $legend-bg: $white;
+// $legend-font-weight: $font-weight-bold;
+// $legend-padding: rem-calc(0 3);
+
+// We use these to style the prefix and postfix input elements
+// $input-prefix-bg: scale-color($white, $lightness: -5%);
+// $input-prefix-border-color: scale-color($white, $lightness: -20%);
+// $input-prefix-border-size: 1px;
+// $input-prefix-border-type: solid;
+// $input-prefix-overflow: hidden;
+// $input-prefix-font-color: $oil;
+// $input-prefix-font-color-alt: $white;
+
+// We use this setting to turn on/off HTML5 number spinners (the up/down arrows)
+// $input-number-spinners: true;
+
+// We use these to style the error states for inputs and labels
+// $input-error-message-padding: rem-calc(6 9 9);
+// $input-error-message-top: -1px;
+// $input-error-message-font-size: rem-calc(12);
+// $input-error-message-font-weight: $font-weight-normal;
+// $input-error-message-font-style: italic;
+// $input-error-message-font-color: $white;
+// $input-error-message-bg-color: $alert-color;
+// $input-error-message-font-color-alt: $oil;
+
+// We use this to style the glowing effect of inputs when focused
+// $input-include-glowing-effect: true;
+// $glowing-effect-fade-time: 0.45s;
+// $glowing-effect-color: $input-focus-border-color;
+
+// Select variables
+// $select-bg-color: $ghost;
+// $select-hover-bg-color: scale-color($select-bg-color, $lightness: -3%);
+
+// 12. Icon Bar
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// We use these to style the icon-bar and items
+// $include-html-icon-bar-classes: $include-html-classes;
+// $icon-bar-bg: $oil;
+// $icon-bar-font-color: $white;
+// $icon-bar-font-color-hover: $icon-bar-font-color;
+// $icon-bar-font-size: 1rem;
+// $icon-bar-hover-color: $primary-color;
+// $icon-bar-icon-color: $white;
+// $icon-bar-icon-color-hover: $icon-bar-icon-color;
+// $icon-bar-icon-size: 1.875rem;
+// $icon-bar-image-width: 1.875rem;
+// $icon-bar-image-height: 1.875rem;
+// $icon-bar-active-color: $primary-color;
+// $icon-bar-item-padding: 1.25rem;
+
+// 13. Inline Lists
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-inline-list-classes: $include-html-classes;
+
+// We use this to control the margins and padding of the inline list.
+// $inline-list-top-margin: 0;
+// $inline-list-opposite-margin: 0;
+// $inline-list-bottom-margin: rem-calc(17);
+// $inline-list-default-float-margin: rem-calc(-22);
+// $inline-list-default-float-list-margin: rem-calc(22);
+
+// $inline-list-padding: 0;
+
+// We use this to control the overflow of the inline list.
+// $inline-list-overflow: hidden;
+
+// We use this to control the list items
+// $inline-list-display: block;
+
+// We use this to control any elements within list items
+// $inline-list-children-display: block;
+
+// 14. Joyride
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-joyride-classes: $include-html-classes;
+
+// Controlling default Joyride styles
+// $joyride-tip-bg: $oil;
+// $joyride-tip-default-width: 300px;
+// $joyride-tip-padding: rem-calc(18 20 24);
+// $joyride-tip-border: solid 1px $charcoal;
+// $joyride-tip-radius: 4px;
+// $joyride-tip-position-offset: 22px;
+
+// Here, we're setting the tip font styles
+// $joyride-tip-font-color: $white;
+// $joyride-tip-font-size: rem-calc(14);
+// $joyride-tip-header-weight: $font-weight-bold;
+
+// This changes the nub size
+// $joyride-tip-nub-size: 10px;
+
+// This adjusts the styles for the timer when its enabled
+// $joyride-tip-timer-width: 50px;
+// $joyride-tip-timer-height: 3px;
+// $joyride-tip-timer-color: $steel;
+
+// This changes up the styles for the close button
+// $joyride-tip-close-color: $monsoon;
+// $joyride-tip-close-size: 24px;
+// $joyride-tip-close-weight: $font-weight-normal;
+
+// When Joyride is filling the screen, we use this style for the bg
+// $joyride-screenfill: rgba(0,0,0,0.5);
+
+// 15. Keystrokes
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-keystroke-classes: $include-html-classes;
+
+// We use these to control text styles.
+// $keystroke-font: "Consolas", "Menlo", "Courier", monospace;
+// $keystroke-font-size: inherit;
+// $keystroke-font-color: $jet;
+// $keystroke-font-color-alt: $white;
+// $keystroke-function-factor: -7%;
+
+// We use this to control keystroke padding.
+// $keystroke-padding: rem-calc(2 4 0);
+
+// We use these to control background and border styles.
+// $keystroke-bg: scale-color($white, $lightness: $keystroke-function-factor);
+// $keystroke-border-style: solid;
+// $keystroke-border-width: 1px;
+// $keystroke-border-color: scale-color($keystroke-bg, $lightness: $keystroke-function-factor);
+// $keystroke-radius: $global-radius;
+
+// 16. Labels
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-label-classes: $include-html-classes;
+
+// We use these to style the labels
+// $label-padding: rem-calc(4 8 4);
+// $label-radius: $global-radius;
+
+// We use these to style the label text
+// $label-font-sizing: rem-calc(11);
+// $label-font-weight: $font-weight-normal;
+// $label-font-color: $oil;
+// $label-font-color-alt: $white;
+// $label-font-family: $body-font-family;
+
+// 17. Magellan
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-magellan-classes: $include-html-classes;
+
+// $magellan-bg: $white;
+// $magellan-padding: 0 !important;
+
+// 18. Off-canvas
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-off-canvas-classes: $include-html-classes;
+
+// $tabbar-bg: $oil;
+// $tabbar-height: rem-calc(45);
+// $tabbar-icon-width: $tabbar-height;
+// $tabbar-line-height: $tabbar-height;
+// $tabbar-color: $white;
+// $tabbar-middle-padding: 0 rem-calc(10);
+
+// Off Canvas Divider Styles
+// $tabbar-right-section-border: solid 1px scale-color($tabbar-bg, $lightness: 13%);
+// $tabbar-left-section-border: solid 1px scale-color($tabbar-bg, $lightness: -50%);
+
+// Off Canvas Tab Bar Headers
+// $tabbar-header-color: $white;
+// $tabbar-header-weight: $font-weight-bold;
+// $tabbar-header-line-height: $tabbar-height;
+// $tabbar-header-margin: 0;
+
+// Off Canvas Menu Variables
+// $off-canvas-width: rem-calc(250);
+// $off-canvas-bg: $oil;
+// $off-canvas-bg-hover: scale-color($tabbar-bg, $lightness: -30%);
+
+// Off Canvas Menu List Variables
+// $off-canvas-label-padding: 0.3rem rem-calc(15);
+// $off-canvas-label-color: $aluminum;
+// $off-canvas-label-text-transform: uppercase;
+// $off-canvas-label-font-size: rem-calc(12);
+// $off-canvas-label-font-weight: $font-weight-bold;
+// $off-canvas-label-bg: $tuatara;
+// $off-canvas-label-border-top: 1px solid scale-color($tuatara, $lightness: 14%);
+// $off-canvas-label-border-bottom: none;
+// $off-canvas-label-margin:0;
+// $off-canvas-link-padding: rem-calc(10, 15);
+// $off-canvas-link-color: rgba($white, 0.7);
+// $off-canvas-link-border-bottom: 1px solid scale-color($off-canvas-bg, $lightness: -25%);
+// $off-canvas-back-bg: $tuatara;
+// $off-canvas-back-border-top: $off-canvas-label-border-top;
+// $off-canvas-back-border-bottom: $off-canvas-label-border-bottom;
+// $off-canvas-back-hover-bg: scale-color($off-canvas-back-bg, $lightness: -30%);
+// $off-canvas-back-hover-border-top: 1px solid scale-color($off-canvas-label-bg, $lightness: 14%);
+// $off-canvas-back-hover-border-bottom: none;
+
+// Off Canvas Menu Icon Variables
+// $tabbar-menu-icon-color: $white;
+// $tabbar-menu-icon-hover: scale-color($tabbar-menu-icon-color, $lightness: -30%);
+
+// $tabbar-menu-icon-text-indent: rem-calc(35);
+// $tabbar-menu-icon-width: $tabbar-height;
+// $tabbar-menu-icon-height: $tabbar-height;
+// $tabbar-menu-icon-padding: 0;
+
+// $tabbar-hamburger-icon-width: rem-calc(16);
+// $tabbar-hamburger-icon-left: false;
+// $tabbar-hamburger-icon-top: false;
+// $tabbar-hamburger-icon-thickness: 1px;
+// $tabbar-hamburger-icon-gap: 6px;
+
+// Off Canvas Back-Link Overlay
+// $off-canvas-overlay-transition: background 300ms ease;
+// $off-canvas-overlay-cursor: pointer;
+// $off-canvas-overlay-box-shadow: -4px 0 4px rgba($black, 0.5), 4px 0 4px rgba($black, 0.5);
+// $off-canvas-overlay-background: rgba($white, 0.2);
+// $off-canvas-overlay-background-hover: rgba($white, 0.05);
+
+// Transition Variables
+// $menu-slide: "transform 500ms ease";
+
+// 19. Orbit
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-orbit-classes: $include-html-classes;
+
+// We use these to control the caption styles
+// $orbit-container-bg: none;
+// $orbit-caption-bg: rgba(51,51,51, 0.8);
+// $orbit-caption-font-color: $white;
+// $orbit-caption-font-size: rem-calc(14);
+// $orbit-caption-position: "bottom"; // Supported values: "bottom", "under"
+// $orbit-caption-padding: rem-calc(10 14);
+// $orbit-caption-height: auto;
+
+// We use these to control the left/right nav styles
+// $orbit-nav-bg: transparent;
+// $orbit-nav-bg-hover: rgba(0,0,0,0.3);
+// $orbit-nav-arrow-color: $white;
+// $orbit-nav-arrow-color-hover: $white;
+
+// We use these to control the timer styles
+// $orbit-timer-bg: rgba(255,255,255,0.3);
+// $orbit-timer-show-progress-bar: true;
+
+// We use these to control the bullet nav styles
+// $orbit-bullet-nav-color: $iron;
+// $orbit-bullet-nav-color-active: $aluminum;
+// $orbit-bullet-radius: rem-calc(9);
+
+// We use these to controls the style of slide numbers
+// $orbit-slide-number-bg: rgba(0,0,0,0);
+// $orbit-slide-number-font-color: $white;
+// $orbit-slide-number-padding: rem-calc(5);
+
+// Hide controls on small
+// $orbit-nav-hide-for-small: true;
+// $orbit-bullet-hide-for-small: true;
+// $orbit-timer-hide-for-small: true;
+
+// Graceful Loading Wrapper and preloader
+// $wrapper-class: "slideshow-wrapper";
+// $preloader-class: "preloader";
+
+// 20. Pagination
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-pagination-classes: $include-html-classes;
+
+// We use these to control the pagination container
+// $pagination-height: rem-calc(24);
+// $pagination-margin: rem-calc(-5);
+
+// We use these to set the list-item properties
+// $pagination-li-float: $default-float;
+// $pagination-li-height: rem-calc(24);
+// $pagination-li-font-color: $jet;
+// $pagination-li-font-size: rem-calc(14);
+// $pagination-li-margin: rem-calc(5);
+
+// We use these for the pagination anchor links
+// $pagination-link-pad: rem-calc(1 10 1);
+// $pagination-link-font-color: $aluminum;
+// $pagination-link-active-bg: scale-color($white, $lightness: -10%);
+
+// We use these for disabled anchor links
+// $pagination-link-unavailable-cursor: default;
+// $pagination-link-unavailable-font-color: $aluminum;
+// $pagination-link-unavailable-bg-active: transparent;
+
+// We use these for currently selected anchor links
+// $pagination-link-current-background: $primary-color;
+// $pagination-link-current-font-color: $white;
+// $pagination-link-current-font-weight: $font-weight-bold;
+// $pagination-link-current-cursor: default;
+// $pagination-link-current-active-bg: $primary-color;
+
+// 21. Panels
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-panel-classes: $include-html-classes;
+
+// We use these to control the background and border styles
+// $panel-bg: scale-color($white, $lightness: -5%);
+// $panel-border-style: solid;
+// $panel-border-size: 1px;
+
+// We use this % to control how much we darken things on hover
+// $panel-function-factor: -11%;
+// $panel-border-color: scale-color($panel-bg, $lightness: $panel-function-factor);
+
+// We use these to set default inner padding and bottom margin
+// $panel-margin-bottom: rem-calc(20);
+// $panel-padding: rem-calc(20);
+
+// We use these to set default font colors
+// $panel-font-color: $oil;
+// $panel-font-color-alt: $white;
+
+// $panel-header-adjust: true;
+// $callout-panel-link-color: $primary-color;
+
+// 22. Pricing Tables
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-pricing-classes: $include-html-classes;
+
+// We use this to control the border color
+// $price-table-border: solid 1px $gainsboro;
+
+// We use this to control the bottom margin of the pricing table
+// $price-table-margin-bottom: rem-calc(20);
+
+// We use these to control the title styles
+// $price-title-bg: $oil;
+// $price-title-padding: rem-calc(15 20);
+// $price-title-align: center;
+// $price-title-color: $smoke;
+// $price-title-weight: $font-weight-normal;
+// $price-title-size: rem-calc(16);
+// $price-title-font-family: $body-font-family;
+
+// We use these to control the price styles
+// $price-money-bg: $vapor;
+// $price-money-padding: rem-calc(15 20);
+// $price-money-align: center;
+// $price-money-color: $oil;
+// $price-money-weight: $font-weight-normal;
+// $price-money-size: rem-calc(32);
+// $price-money-font-family: $body-font-family;
+
+// We use these to control the description styles
+// $price-bg: $white;
+// $price-desc-color: $monsoon;
+// $price-desc-padding: rem-calc(15);
+// $price-desc-align: center;
+// $price-desc-font-size: rem-calc(12);
+// $price-desc-weight: $font-weight-normal;
+// $price-desc-line-height: 1.4;
+// $price-desc-bottom-border: dotted 1px $gainsboro;
+
+// We use these to control the list item styles
+// $price-item-color: $oil;
+// $price-item-padding: rem-calc(15);
+// $price-item-align: center;
+// $price-item-font-size: rem-calc(14);
+// $price-item-weight: $font-weight-normal;
+// $price-item-bottom-border: dotted 1px $gainsboro;
+
+// We use these to control the CTA area styles
+// $price-cta-bg: $white;
+// $price-cta-align: center;
+// $price-cta-padding: rem-calc(20 20 0);
+
+// 23. Progress Bar
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-media-classes: $include-html-classes;
+
+// We use this to set the progress bar height
+// $progress-bar-height: rem-calc(25);
+// $progress-bar-color: $vapor  ;
+
+// We use these to control the border styles
+// $progress-bar-border-color: scale-color($white, $lightness: 20%);
+// $progress-bar-border-size: 1px;
+// $progress-bar-border-style: solid;
+// $progress-bar-border-radius: $global-radius;
+
+// We use these to control the margin & padding
+// $progress-bar-pad: rem-calc(2);
+// $progress-bar-margin-bottom: rem-calc(10);
+
+// We use these to set the meter colors
+// $progress-meter-color: $primary-color;
+// $progress-meter-secondary-color: $secondary-color;
+// $progress-meter-success-color: $success-color;
+// $progress-meter-alert-color: $alert-color;
+
+// 24. Range Slider
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-range-slider-classes: $include-html-classes;
+
+// These variables define the slider bar styles
+// $range-slider-bar-width: 100%;
+// $range-slider-bar-height: rem-calc(16);
+
+// $range-slider-bar-border-width: 1px;
+// $range-slider-bar-border-style: solid;
+// $range-slider-bar-border-color: $gainsboro;
+// $range-slider-radius: $global-radius;
+// $range-slider-round: $global-rounded;
+// $range-slider-bar-bg-color: $ghost;
+
+// Vertical bar styles
+// $range-slider-vertical-bar-width: rem-calc(16);
+// $range-slider-vertical-bar-height: rem-calc(200);
+
+// These variables define the slider handle styles
+// $range-slider-handle-width: rem-calc(32);
+// $range-slider-handle-height: rem-calc(22);
+// $range-slider-handle-position-top: rem-calc(-5);
+// $range-slider-handle-bg-color: $primary-color;
+// $range-slider-handle-border-width: 1px;
+// $range-slider-handle-border-style: solid;
+// $range-slider-handle-border-color: none;
+// $range-slider-handle-radius: $global-radius;
+// $range-slider-handle-round: $global-rounded;
+// $range-slider-handle-bg-hover-color: scale-color($primary-color, $lightness: -12%);
+// $range-slider-handle-cursor: pointer;
+
+// 25. Reveal
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-reveal-classes: $include-html-classes;
+
+// We use these to control the style of the reveal overlay.
+// $reveal-overlay-bg: rgba($black, .45);
+// $reveal-overlay-bg-old: $black;
+
+// We use these to control the style of the modal itself.
+// $reveal-modal-bg: $white;
+// $reveal-position-top: rem-calc(100);
+// $reveal-default-width: 80%;
+// $reveal-max-width: $row-width;
+// $reveal-modal-padding: rem-calc(20);
+// $reveal-box-shadow: 0 0 10px rgba($black,.4);
+
+// We use these to style the reveal close button
+// $reveal-close-font-size: rem-calc(40);
+// $reveal-close-top: rem-calc(8);
+// $reveal-close-side: rem-calc(11);
+// $reveal-close-color: $base;
+// $reveal-close-weight: $font-weight-bold;
+
+// We use this to set the default radius used throughout the core.
+// $reveal-radius: $global-radius;
+// $reveal-round: $global-rounded;
+
+// We use these to control the modal border
+// $reveal-border-style: solid;
+// $reveal-border-width: 1px;
+// $reveal-border-color: $steel;
+
+// $reveal-modal-class: "reveal-modal";
+// $close-reveal-modal-class: "close-reveal-modal";
+
+// 26. Side Nav
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-nav-classes: $include-html-classes;
+
+// We use this to control padding.
+// $side-nav-padding: rem-calc(14 0);
+
+// We use these to control list styles.
+// $side-nav-list-type: none;
+// $side-nav-list-position: inside;
+// $side-nav-list-margin: rem-calc(0 0 7 0);
+
+// We use these to control link styles.
+// $side-nav-link-color: $primary-color;
+// $side-nav-link-color-active: scale-color($side-nav-link-color, $lightness: 30%);
+// $side-nav-link-color-hover: scale-color($side-nav-link-color, $lightness: 30%);
+// $side-nav-link-bg-hover: hsla(0, 0, 0, 0.025);
+// $side-nav-link-margin: 0;
+// $side-nav-link-padding: rem-calc(7 14);
+// $side-nav-font-size: rem-calc(14);
+// $side-nav-font-weight: $font-weight-normal;
+// $side-nav-font-weight-active: $side-nav-font-weight;
+// $side-nav-font-family: $body-font-family;
+// $side-nav-font-family-active: $side-nav-font-family;
+
+// We use these to control heading styles.
+// $side-nav-heading-color: $side-nav-link-color;
+// $side-nav-heading-font-size: $side-nav-font-size;
+// $side-nav-heading-font-weight: bold;
+// $side-nav-heading-text-transform: uppercase;
+
+// We use these to control border styles
+// $side-nav-divider-size: 1px;
+// $side-nav-divider-style: solid;
+// $side-nav-divider-color: scale-color($white, $lightness: 10%);
+
+// 27. Split Buttons
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-button-classes: $include-html-classes;
+
+// We use these to control different shared styles for Split Buttons
+// $split-button-function-factor: 10%;
+// $split-button-pip-color: $white;
+// $split-button-pip-color-alt: $oil;
+// $split-button-active-bg-tint: rgba(0,0,0,0.1);
+
+// We use these to control tiny split buttons
+// $split-button-padding-tny: $button-pip-tny * 10;
+// $split-button-span-width-tny: $button-pip-tny * 6;
+// $split-button-pip-size-tny: $button-pip-tny;
+// $split-button-pip-top-tny: $button-pip-tny * 2;
+// $split-button-pip-default-float-tny: rem-calc(-6);
+
+// We use these to control small split buttons
+// $split-button-padding-sml: $button-pip-sml * 10;
+// $split-button-span-width-sml: $button-pip-sml * 6;
+// $split-button-pip-size-sml: $button-pip-sml;
+// $split-button-pip-top-sml: $button-pip-sml * 1.5;
+// $split-button-pip-default-float-sml: rem-calc(-6);
+
+// We use these to control medium split buttons
+// $split-button-padding-med: $button-pip-med * 9;
+// $split-button-span-width-med: $button-pip-med * 5.5;
+// $split-button-pip-size-med: $button-pip-med - rem-calc(3);
+// $split-button-pip-top-med: $button-pip-med * 1.5;
+// $split-button-pip-default-float-med: rem-calc(-6);
+
+// We use these to control large split buttons
+// $split-button-padding-lrg: $button-pip-lrg * 8;
+// $split-button-span-width-lrg: $button-pip-lrg * 5;
+// $split-button-pip-size-lrg: $button-pip-lrg - rem-calc(6);
+// $split-button-pip-top-lrg: $button-pip-lrg + rem-calc(5);
+// $split-button-pip-default-float-lrg: rem-calc(-6);
+
+// 28. Sub Nav
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-nav-classes: $include-html-classes;
+
+// We use these to control margin and padding
+// $sub-nav-list-margin: rem-calc(-4 0 18);
+// $sub-nav-list-padding-top: rem-calc(4);
+
+// We use this to control the definition
+// $sub-nav-font-family: $body-font-family;
+// $sub-nav-font-size: rem-calc(14);
+// $sub-nav-font-color: $aluminum;
+// $sub-nav-font-weight: $font-weight-normal;
+// $sub-nav-text-decoration: none;
+// $sub-nav-padding: rem-calc(3 16);
+// $sub-nav-border-radius: 3px;
+// $sub-nav-font-color-hover: scale-color($sub-nav-font-color, $lightness: -25%);
+
+// We use these to control the active item styles
+// $sub-nav-active-font-weight: $font-weight-normal;
+// $sub-nav-active-bg: $primary-color;
+// $sub-nav-active-bg-hover: scale-color($sub-nav-active-bg, $lightness: -14%);
+// $sub-nav-active-color: $white;
+// $sub-nav-active-padding: $sub-nav-padding;
+// $sub-nav-active-cursor: default;
+
+// $sub-nav-item-divider: "";
+// $sub-nav-item-divider-margin: rem-calc(12);
+
+// 29. Switch
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-form-classes: $include-html-classes;
+
+// Controlling border styles and background colors for the switch container
+// $switch-border-color: scale-color($white, $lightness: -20%);
+// $switch-border-style: solid;
+// $switch-border-width: 1px;
+// $switch-bg: $white;
+
+// We use these to control the switch heights for our default classes
+// $switch-height-tny: rem-calc(22);
+// $switch-height-sml: rem-calc(28);
+// $switch-height-med: rem-calc(36);
+// $switch-height-lrg: rem-calc(44);
+// $switch-bottom-margin: rem-calc(20);
+
+// We use these to control default font sizes for our classes.
+// $switch-font-size-tny: 11px;
+// $switch-font-size-sml: 12px;
+// $switch-font-size-med: 14px;
+// $switch-font-size-lrg: 17px;
+// $switch-label-side-padding: 6px;
+
+// We use these to style the switch-paddle
+// $switch-paddle-bg: $white;
+// $switch-paddle-fade-to-color: scale-color($switch-paddle-bg, $lightness: -10%);
+// $switch-paddle-border-color: scale-color($switch-paddle-bg, $lightness: -35%);
+// $switch-paddle-border-width: 1px;
+// $switch-paddle-border-style: solid;
+// $switch-paddle-transition-speed: .1s;
+// $switch-paddle-transition-ease: ease-out;
+// $switch-positive-color: scale-color($success-color, $lightness: 94%);
+// $switch-negative-color: $white-smoke;
+
+// Outline Style for tabbing through switches
+// $switch-label-outline: 1px dotted $jumbo;
+
+// 30. Tables
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-table-classes: $include-html-classes;
+
+// These control the background color for the table and even rows
+// $table-bg: $white;
+// $table-even-row-bg: $snow;
+
+// These control the table cell border style
+// $table-border-style: solid;
+// $table-border-size: 1px;
+// $table-border-color: $gainsboro;
+
+// These control the table head styles
+// $table-head-bg: $white-smoke  ;
+// $table-head-font-size: rem-calc(14);
+// $table-head-font-color: $jet;
+// $table-head-font-weight: $font-weight-bold;
+// $table-head-padding: rem-calc(8 10 10);
+
+// These control the row padding and font styles
+// $table-row-padding: rem-calc(9 10);
+// $table-row-font-size: rem-calc(14);
+// $table-row-font-color: $jet;
+// $table-line-height: rem-calc(18);
+
+// These are for controlling the layout, display and margin of tables
+// $table-layout: auto;
+// $table-display: table-cell;
+// $table-margin-bottom: rem-calc(20);
+
+// 31. Tabs
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-tabs-classes: $include-html-classes;
+
+// $tabs-navigation-padding: rem-calc(16);
+// $tabs-navigation-bg-color: $silver  ;
+// $tabs-navigation-active-bg-color: $white;
+// $tabs-navigation-hover-bg-color: scale-color($tabs-navigation-bg-color, $lightness: -6%);
+// $tabs-navigation-font-color: $jet;
+// $tabs-navigation-active-font-color: $tabs-navigation-font-color;
+// $tabs-navigation-font-size: rem-calc(16);
+// $tabs-navigation-font-family: $body-font-family;
+
+// $tabs-content-margin-bottom: rem-calc(24);
+// $tabs-content-padding: ($column-gutter/2);
+
+// $tabs-vertical-navigation-margin-bottom: 1.25rem;
+
+// 32. Thumbnails
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-media-classes: $include-html-classes;
+
+// We use these to control border styles
+// $thumb-border-style: solid;
+// $thumb-border-width: 4px;
+// $thumb-border-color: $white;
+// $thumb-box-shadow: 0 0 0 1px rgba($black,.2);
+// $thumb-box-shadow-hover: 0 0 6px 1px rgba($primary-color,0.5);
+
+// Radius and transition speed for thumbs
+// $thumb-radius: $global-radius;
+// $thumb-transition-speed: 200ms;
+
+// 33. Tooltips
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-tooltip-classes: $include-html-classes;
+
+// $has-tip-border-bottom: dotted 1px $iron;
+// $has-tip-font-weight: $font-weight-bold;
+// $has-tip-font-color: $oil;
+// $has-tip-border-bottom-hover: dotted 1px scale-color($primary-color, $lightness: -55%);
+// $has-tip-font-color-hover: $primary-color;
+// $has-tip-cursor-type: help;
+
+// $tooltip-padding: rem-calc(12);
+// $tooltip-bg: $oil;
+// $tooltip-font-size: rem-calc(14);
+// $tooltip-font-weight: $font-weight-normal;
+// $tooltip-font-color: $white;
+// $tooltip-line-height: 1.3;
+// $tooltip-close-font-size: rem-calc(10);
+// $tooltip-close-font-weight: $font-weight-normal;
+// $tooltip-close-font-color: $monsoon;
+// $tooltip-font-size-sml: rem-calc(14);
+// $tooltip-radius: $global-radius;
+// $tooltip-rounded: $global-rounded;
+// $tooltip-pip-size: 5px;
+// $tooltip-max-width: 300px;
+
+// 34. Top Bar
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-top-bar-classes: $include-html-classes;
+
+// Background color for the top bar
+// $topbar-bg-color: $oil;
+$topbar-bg-color: #7299ac;
+// $topbar-bg: $topbar-bg-color;
+
+// Height and margin
+// $topbar-height: rem-calc(45);
+// $topbar-margin-bottom: 0;
+
+// Controlling the styles for the title in the top bar
+// $topbar-title-weight: $font-weight-normal;
+// $topbar-title-font-size: rem-calc(17);
+
+// Style the top bar dropdown elements
+// $topbar-dropdown-bg: $oil;
+$topbar-dropdown-bg: $topbar-bg-color;
+// $topbar-dropdown-link-color: $white;
+$topbar-dropdown-link-color-hover: #eee;
+// $topbar-dropdown-link-bg: $oil;
+$topbar-dropdown-link-bg: $topbar-bg-color;
+// $topbar-dropdown-link-bg-hover: $oil;
+$topbar-dropdown-link-bg-hover: $topbar-bg-color;
+// $topbar-dropdown-link-weight: $font-weight-normal;
+// $topbar-dropdown-toggle-size: 5px;
+// $topbar-dropdown-toggle-color: $white;
+// $topbar-dropdown-toggle-alpha: 0.4;
+
+// Set the link colors and styles for top-level nav
+// $topbar-link-color: $white;
+// $topbar-link-color-hover: $white;
+// $topbar-link-color-active: $white;
+// $topbar-link-color-active-hover: $white;
+// $topbar-link-weight: $font-weight-normal;
+// $topbar-link-font-size: rem-calc(13);
+$topbar-link-weight: bold;
+// $topbar-link-hover-lightness: -10%; // Darken by 10%
+// $topbar-link-bg: $topbar-bg;
+// $topbar-link-bg-color-hover: $charcoal;
+$topbar-link-bg-color-hover: scale-color($topbar-bg-color, $lightness: -14%);
+// $topbar-link-bg-hover: $oil;
+$topbar-link-bg-hover: scale-color($topbar-bg-color, $lightness: -14%);
+// $topbar-link-bg-active: $primary-color;
+$topbar-link-bg-active: $topbar-bg-color;
+// $topbar-link-bg-active-hover: scale-color($primary-color, $lightness: -14%);
+$topbar-link-bg-active-hover: scale-color($topbar-bg-color, $lightness: -14%);
+// $topbar-link-font-family: $body-font-family;
+// $topbar-link-text-transform: none;
+// $topbar-link-padding: ($topbar-height / 3);
+// $topbar-back-link-size: $h5-font-size;
+// $topbar-link-dropdown-padding: 20px;
+
+// $topbar-button-font-size: 0.75rem;
+// $topbar-button-top: 7px;
+
+// $topbar-dropdown-label-color: $monsoon;
+// $topbar-dropdown-label-text-transform: uppercase;
+$topbar-dropdown-label-font-weight: $font-weight-bold;
+// $topbar-dropdown-label-font-size: rem-calc(10);
+// $topbar-dropdown-label-bg: $oil;
+
+// Top menu icon styles
+// $topbar-menu-link-transform: uppercase;
+// $topbar-menu-link-font-size: rem-calc(13);
+$topbar-menu-link-weight: $font-weight-bold;
+// $topbar-menu-link-color: $white;
+// $topbar-menu-icon-color: $white;
+// $topbar-menu-link-color-toggled: $jumbo;
+// $topbar-menu-icon-color-toggled: $jumbo;
+
+// Transitions and breakpoint styles
+// $topbar-transition-speed: 300ms;
+// Using rem-calc for the below breakpoint causes issues with top bar
+// $topbar-breakpoint: #{lower-bound($medium-range)}; // Change to 9999px for always mobile layout
+// $topbar-media-query: $medium-up;
+
+// Divider Styles
+// $topbar-divider-border-bottom: solid 1px  scale-color($topbar-bg-color, $lightness: 13%);
+// $topbar-divider-border-top: solid 1px scale-color($topbar-bg-color, $lightness: -50%);
+
+// Sticky Class
+// $topbar-sticky-class: ".sticky";
+// $topbar-arrows: true; //Set false to remove the triangle icon from the menu item
+
+// 36. Visibility Classes
+// - - - - - - - - - - - - - - - - - - - - - - - - -
+
+// $include-html-visibility-classes: $include-html-classes;
+// $include-table-visibility-classes: true;
+// $include-legacy-visibility-classes: true;
+// $include-accessibility-classes: true;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/scss/app.scss
----------------------------------------------------------------------
diff --git a/nifi-site/src/scss/app.scss b/nifi-site/src/scss/app.scss
new file mode 100644
index 0000000..935ef74
--- /dev/null
+++ b/nifi-site/src/scss/app.scss
@@ -0,0 +1,177 @@
+@import "settings";
+@import "foundation/components/type";
+@import "foundation/components/grid";
+@import "foundation/components/top-bar";
+@import "foundation/components/tables";
+
+// @import "foundation";
+// Or selectively include components
+// @import
+//   "foundation/components/accordion",
+//   "foundation/components/alert-boxes",
+//   "foundation/components/block-grid",
+//   "foundation/components/breadcrumbs",
+//   "foundation/components/button-groups",
+//   "foundation/components/buttons",
+//   "foundation/components/clearing",
+//   "foundation/components/dropdown",
+//   "foundation/components/dropdown-buttons",
+//   "foundation/components/flex-video",
+//   "foundation/components/forms",
+//   "foundation/components/grid",
+//   "foundation/components/inline-lists",
+//   "foundation/components/joyride",
+//   "foundation/components/keystrokes",
+//   "foundation/components/labels",
+//   "foundation/components/magellan",
+//   "foundation/components/orbit",
+//   "foundation/components/pagination",
+//   "foundation/components/panels",
+//   "foundation/components/pricing-tables",
+//   "foundation/components/progress-bars",
+//   "foundation/components/reveal",
+//   "foundation/components/side-nav",
+//   "foundation/components/split-buttons",
+//   "foundation/components/sub-nav",
+//   "foundation/components/switches",
+//   "foundation/components/tables",
+//   "foundation/components/tabs",
+//   "foundation/components/thumbs",
+//   "foundation/components/tooltips",
+//   "foundation/components/top-bar",
+//   "foundation/components/type",
+//   "foundation/components/offcanvas",
+//   "foundation/components/visibility";
+
+html, html a {
+    -webkit-font-smoothing: antialiased;
+    text-shadow: 1px 1px 1px rgba(0,0,0,0.004);
+}
+
+.wf-loading h1, .wf-loading p {
+    visibility:hidden;
+}
+
+div.large-space {
+    height: 3em;
+}
+
+div.medium-space {
+    height: 2em;
+}
+
+div.small-space {
+    height: 1em;
+}
+
+#logo-top-bar {
+    max-width: 27px;
+    max-height: 40px;
+    margin-top: -2px;
+}
+
+#logo-top-bar:hover {
+    -webkit-filter: brightness(1.03);
+    filter: brightness(1.03);
+}
+
+#nifi-landing {
+    font-size: 4.5em;
+}
+
+.nifi-txt {
+    font-family: 'Oswald', sans-serif;
+    font-weight: 500;
+}
+
+span.nifi-txt {
+    font-size: larger;
+}
+
+.ni {
+    color: #7A96A4;
+}
+
+.fi {
+    color: #0F3541;
+}
+
+.right-text {
+    text-align: right;
+}
+
+p.description {
+    font-size: 1.25em;
+    font-weight: 200;
+    color: #333;
+}
+
+#flow {
+    -moz-box-shadow: 0 0 25px #0F3541;
+    -webkit-box-shadow: 0 0 25px #0F3541;
+    box-shadow: 0px 0px 25px #0F3541;
+}
+
+div.features {
+    background-color: #D9E4E8;
+}
+
+#features-content ul {
+    padding-left: 10px;
+}
+
+i.external-link {
+    margin-right: 5px;
+}
+
+/*
+    Tables
+*/
+
+table {
+    background-color: #fefefe;
+    border: 1px solid #ccc;
+    border-left: 6px solid #ccc;
+    color: #555;
+    display: block;
+    margin-bottom: 12px;
+    padding: 5px 8px;
+}
+
+tr td {
+    font-size: 14px;
+    vertical-align:top;
+    text-align:left;
+    padding: 4px;
+    border-width: 0;
+}
+
+tr th {
+    font-size: 16px;
+    vertical-align:top;
+    text-align:left;
+    padding: 4px;
+    border-width: 0;
+    white-space: nowrap;
+}
+
+table tr:nth-of-type(even) {
+    background-color: transparent !important;
+}
+
+/*
+    Footer
+*/
+
+div.footer {
+    margin-top: 2em;
+    text-align: center;
+}
+
+div.footer p {
+    font-size: .7em;
+}
+
+div.clear {
+    clear: both;
+}
\ No newline at end of file


[02/36] incubator-nifi git commit: NIFI-305: Minor documentation update

Posted by mc...@apache.org.
NIFI-305: Minor documentation update


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

Branch: refs/heads/NIFI-250
Commit: ad40903458cd2bbf98284d693a057519d8bbf775
Parents: 98afcce
Author: gresockj <jg...@gmail.com>
Authored: Tue Jan 27 17:51:16 2015 -0500
Committer: gresockj <jg...@gmail.com>
Committed: Tue Jan 27 17:51:16 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ad409034/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
index 7846c7d..b838d51 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
@@ -48,7 +48,7 @@ import org.apache.nifi.processors.standard.util.BinManager;
 import org.apache.nifi.processors.standard.util.FlowFileSessionWrapper;
 
 /**
- * Base class for MergeContent.
+ * Base class for file-binning processors, including MergeContent.
  *
  */
 public abstract class BinFiles extends AbstractSessionFactoryProcessor {


[29/36] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: e45dea1b1cb79b1b28e20241fb0e35130b33b2a3
Parents: 8140b2a 7724dad
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Feb 3 22:44:33 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Feb 3 22:44:33 2015 -0500

----------------------------------------------------------------------
 .../nifi/util/StandardProcessorTestRunner.java  |   7 +-
 .../util/TestStandardProcessorTestRunner.java   |   2 +
 .../nifi/processors/standard/PostHTTP.java      |  30 ++-
 .../processors/standard/CaptureServlet.java     |  54 ++++
 .../nifi/processors/standard/TestPostHTTP.java  | 249 +++++++++++++++++++
 .../nifi/processors/standard/TestServer.java    |  27 +-
 6 files changed, 348 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[34/36] incubator-nifi git commit: NIFI-317: - Code clean up. Earlier versions of this used a GET request which required the img be url encoded. Converted to using POST due to URL length limitations but left the encoding code in place.

Posted by mc...@apache.org.
NIFI-317:
- Code clean up. Earlier versions of this used a GET request which required the img be url encoded. Converted to using POST due to URL length limitations but left the encoding code in place.

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

Branch: refs/heads/NIFI-250
Commit: ed53b46b7aac460d0b832c20332c8881419c4fab
Parents: 3a0cb2d
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 4 10:02:51 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 4 10:02:51 2015 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/web/servlet/DownloadSvg.java   | 8 ++------
 .../nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js  | 8 +++-----
 .../main/webapp/js/nf/provenance/nf-provenance-lineage.js    | 4 ++--
 3 files changed, 7 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ed53b46b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
index ab76d4e..d912998 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
@@ -18,7 +18,6 @@ package org.apache.nifi.web.servlet;
 
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.net.URLDecoder;
 import javax.servlet.ServletException;
 import javax.servlet.annotation.WebServlet;
 import javax.servlet.http.HttpServlet;
@@ -44,10 +43,10 @@ public class DownloadSvg extends HttpServlet {
      */
     @Override
     protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
-        final String rawSvg = request.getParameter("svg");
+        final String svg = request.getParameter("svg");
 
         // ensure the image markup has been included
-        if (rawSvg == null) {
+        if (svg == null) {
             // set the response status
             response.setContentType("text/plain");
             response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
@@ -59,9 +58,6 @@ public class DownloadSvg extends HttpServlet {
         }
 
         try {
-            // get the svg and decode it, +'s need to be converted
-            final String svg = URLDecoder.decode(rawSvg.replace("+", "%2B"), "UTF-8");
-
             if (logger.isDebugEnabled()) {
                 logger.debug(svg);
             }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ed53b46b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
index 171b96f..b49a97b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
@@ -406,21 +406,19 @@ nf.Common = {
     }()),
     
     /**
-     * Creates a form inline in order to submit the specified params to the specified URL
-     * using the specified method.
+     * Creates a form inline in order to post the specified params to the specified URL.
      * 
-     * @param {string} method       The method to use
      * @param {string} url          The URL
      * @param {object} params       An object with the params to include in the submission
      */
-    submit: function (method, url, params) {
+    post: function (url, params) {
         // temporarily override beforeunload
         var previousBeforeUnload = window.onbeforeunload;
         window.onbeforeunload = null;
 
         // create a form for submission
         var form = $('<form></form>').attr({
-            'method': method,
+            'method': 'POST',
             'action': url,
             'style': 'display: none;'
         });

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ed53b46b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
index 03b08f0..583fe7e 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
@@ -1287,9 +1287,9 @@ nf.ProvenanceLineage = (function () {
                 svg = '<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">\n' + svg;
 
                 // send to server to initiate download... client side only support is too browser specific at this point
-                nf.Common.submit('POST', './download-svg', {
+                nf.Common.post('./download-svg', {
                     'filename': 'provenance',
-                    'svg': encodeURIComponent(svg)
+                    'svg': svg
                 });
             });
 


[18/36] incubator-nifi git commit: NIFI-291 completed incorporation of Elser feedback

Posted by mc...@apache.org.
NIFI-291 completed incorporation of Elser feedback


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

Branch: refs/heads/NIFI-250
Commit: d926aca951b8169b0c32a583a178dedb6e1dc605
Parents: 2c2d39c
Author: joewitt <jo...@apache.org>
Authored: Tue Feb 3 09:41:27 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Tue Feb 3 09:41:27 2015 -0500

----------------------------------------------------------------------
 README.md                       | 14 ++++++++++++++
 nifi-nar-maven-plugin/README.md | 14 ++++++++++++++
 nifi-nar-maven-plugin/pom.xml   |  6 +++---
 nifi-site/README.md             | 14 ++++++++++++++
 nifi/README.md                  | 14 ++++++++++++++
 nifi/pom.xml                    | 16 ++++++++--------
 6 files changed, 67 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d926aca9/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index c901e36..7bbc4dd 100644
--- a/README.md
+++ b/README.md
@@ -1,3 +1,17 @@
+<!--
+  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.
+-->
 # Apache NiFi
 
 Apache NiFi is a dataflow system based on the concepts of flow-based programming. It is currently apart of the Apache Incubator.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d926aca9/nifi-nar-maven-plugin/README.md
----------------------------------------------------------------------
diff --git a/nifi-nar-maven-plugin/README.md b/nifi-nar-maven-plugin/README.md
index 941de2b..1b554d9 100644
--- a/nifi-nar-maven-plugin/README.md
+++ b/nifi-nar-maven-plugin/README.md
@@ -1,3 +1,17 @@
+<!--
+  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.
+-->
 # Apache NiFi NAR Maven Plugin
 
 Apache NiFi NAR Maven Plugin helps to build NiFi Archive bundles to support the classloader isolation model of NiFi.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d926aca9/nifi-nar-maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-maven-plugin/pom.xml b/nifi-nar-maven-plugin/pom.xml
index 79e437e..6f46397 100644
--- a/nifi-nar-maven-plugin/pom.xml
+++ b/nifi-nar-maven-plugin/pom.xml
@@ -206,9 +206,9 @@
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
                     <excludes>
-                        <exclude>nb-configuration.xml</exclude>
-                        <exclude>nbactions.xml</exclude>
-                        <exclude>DEPENDENCIES</exclude>
+                        <exclude>nb-configuration.xml</exclude> <!-- convenience exclude for netbeans users -->
+                        <exclude>nbactions.xml</exclude> <!-- convenience excludes for netbeans users -->
+                        <exclude>DEPENDENCIES</exclude> <!-- auto generated by maven while building sources zip and more -->
                     </excludes>
                 </configuration>
             </plugin>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d926aca9/nifi-site/README.md
----------------------------------------------------------------------
diff --git a/nifi-site/README.md b/nifi-site/README.md
index 45276e1..05cabd0 100644
--- a/nifi-site/README.md
+++ b/nifi-site/README.md
@@ -1,3 +1,17 @@
+<!--
+  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.
+-->
 # About
 [Apache NiFi project] (http://nifi.incubator.apache.org).
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d926aca9/nifi/README.md
----------------------------------------------------------------------
diff --git a/nifi/README.md b/nifi/README.md
index e077c0c..70bf7ef 100644
--- a/nifi/README.md
+++ b/nifi/README.md
@@ -1,3 +1,17 @@
+<!--
+  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.
+-->
 # Apache NiFi
 
 Apache NiFi is a dataflow system based on the concepts of flow-based programming. It is currently apart of the Apache Incubator.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d926aca9/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 499ffd6..fb73f9c 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -956,15 +956,15 @@
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
                     <excludes>
-                        <exclude>src/main/webapp/js/json2.js</exclude>
-                        <exclude>src/main/webapp/js/jquery/</exclude>
-                        <exclude>src/main/webapp/js/d3/d3.min.js</exclude>
-                        <exclude>src/main/webapp/js/codemirror/</exclude>
-                        <exclude>src/main/webapp/css/reset.css</exclude>
+                        <exclude>src/main/webapp/js/json2.js</exclude> <!-- license/notice accounted for -->
+                        <exclude>src/main/webapp/js/jquery/</exclude> <!-- license/notice accounted for -->
+                        <exclude>src/main/webapp/js/d3/d3.min.js</exclude> <!-- license/notice accounted for -->
+                        <exclude>src/main/webapp/js/codemirror/</exclude> <!-- license/notice accounted for -->
+                        <exclude>src/main/webapp/css/reset.css</exclude> <!-- license/notice accounted for -->
                         <exclude>src/test/resources/</exclude> <!-- Test datasets -->
-                        <exclude>nb-configuration.xml</exclude>
-                        <exclude>nbactions.xml</exclude>
-                        <exclude>DEPENDENCIES</exclude>
+                        <exclude>nb-configuration.xml</exclude><!-- courtesy exclude for netbeans users -->
+                        <exclude>nbactions.xml</exclude> <!-- courtesy exclude for netbeans users -->
+                        <exclude>DEPENDENCIES</exclude> <!-- auto generated file by maven while building sources.zip -->
                     </excludes>
                 </configuration>
             </plugin>


[27/36] incubator-nifi git commit: Updating nifi-processor-bundle-archetype to use new annotation package names

Posted by mc...@apache.org.
Updating nifi-processor-bundle-archetype to use new annotation package names


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

Branch: refs/heads/NIFI-250
Commit: bbb7f8d46c9b5df61747a1dfd28248b99ec00e9a
Parents: 7724dad
Author: bbende <bb...@gmail.com>
Authored: Tue Feb 3 22:24:32 2015 -0500
Committer: bbende <bb...@gmail.com>
Committed: Tue Feb 3 22:24:32 2015 -0500

----------------------------------------------------------------------
 .../src/main/java/MyProcessor.java                        | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/bbb7f8d4/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java
index 4f2ecff..7d29b5e 100644
--- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java
+++ b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java
@@ -20,9 +20,9 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.*;
-import org.apache.nifi.processor.annotation.CapabilityDescription;
-import org.apache.nifi.processor.annotation.OnScheduled;
-import org.apache.nifi.processor.annotation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -80,9 +80,9 @@ public class MyProcessor extends AbstractProcessor {
 		if ( flowFile == null ) {
 			return;
 		}
-		
+
         // TODO implement
-		
+
     }
 
 }


[16/36] incubator-nifi git commit: NIFI-269 This closes #16. Merge branch 'develop' of https://github.com/davidmoravek/incubator-nifi into develop

Posted by mc...@apache.org.
NIFI-269 This closes #16.  Merge branch 'develop' of https://github.com/davidmoravek/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: fbfebf622d45d0ae972b9e1c47c617afdb8dc978
Parents: 9c61bdb 51b34a0
Author: joewitt <jo...@apache.org>
Authored: Tue Feb 3 07:59:50 2015 -0500
Committer: joewitt <jo...@apache.org>
Committed: Tue Feb 3 07:59:50 2015 -0500

----------------------------------------------------------------------
 nifi/nifi-assembly/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[11/36] incubator-nifi git commit: NIFI-289: - Added logging when error conditions occur in UpdateAttribute REST endpoints.

Posted by mc...@apache.org.
NIFI-289:
- Added logging when error conditions occur in UpdateAttribute REST endpoints.

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

Branch: refs/heads/NIFI-250
Commit: 93b78a7dd5d4e4fa43fc84c5992010389638d595
Parents: 600a47b
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 14:02:24 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 14:02:24 2015 -0500

----------------------------------------------------------------------
 .../nifi/update/attributes/api/RuleResource.java    | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/93b78a7d/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/src/main/java/org/apache/nifi/update/attributes/api/RuleResource.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/src/main/java/org/apache/nifi/update/attributes/api/RuleResource.java b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/src/main/java/org/apache/nifi/update/attributes/api/RuleResource.java
index 05ed0a1..dc2d27c 100644
--- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/src/main/java/org/apache/nifi/update/attributes/api/RuleResource.java
+++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/src/main/java/org/apache/nifi/update/attributes/api/RuleResource.java
@@ -70,6 +70,8 @@ import com.sun.jersey.api.NotFoundException;
 
 import org.apache.nifi.update.attributes.FlowFilePolicy;
 import org.apache.nifi.update.attributes.entity.EvaluationContextEntity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *
@@ -77,6 +79,8 @@ import org.apache.nifi.update.attributes.entity.EvaluationContextEntity;
 @Path("/criteria")
 public class RuleResource {
 
+    private static final Logger logger = LoggerFactory.getLogger(RuleResource.class);
+
     @Context
     private ServletContext servletContext;
 
@@ -612,7 +616,9 @@ public class RuleResource {
         } catch (final InvalidRevisionException ire) {
             throw new WebApplicationException(invalidRevision(ire.getMessage()));
         } catch (final Exception e) {
-            throw new WebApplicationException(error(e.getMessage()));
+            final String message = String.format("Unable to get UpdateAttribute[id=%s] criteria: %s", contextConfig.getProcessorId(), e);
+            logger.error(message, e);
+            throw new WebApplicationException(error(message));
         }
 
         Criteria criteria = null;
@@ -620,7 +626,9 @@ public class RuleResource {
             try {
                 criteria = CriteriaSerDe.deserialize(processorInfo.getAnnotationData());
             } catch (final IllegalArgumentException iae) {
-                throw new WebApplicationException(error("Unable to load existing rules. Deserialization error: " + iae.getMessage()));
+                final String message = String.format("Unable to deserialize existing rules for UpdateAttribute[id=%s]. Deserialization error: %s", contextConfig.getProcessorId(), iae);
+                logger.error(message, iae);
+                throw new WebApplicationException(error(message));
             }
         }
         // ensure the criteria isn't null
@@ -644,7 +652,9 @@ public class RuleResource {
         } catch (final InvalidRevisionException ire) {
             throw new WebApplicationException(invalidRevision(ire.getMessage()));
         } catch (final Exception e) {
-            throw new WebApplicationException(error(e.getMessage()));
+            final String message = String.format("Unable to save UpdateAttribute[id=%s] criteria: %s", contextConfig.getProcessorId(), e);
+            logger.error(message, e);
+            throw new WebApplicationException(error(message));
         }
     }
 


[33/36] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: 3a0cb2d63404210da1b4b6827bec84f87166f60b
Parents: 4e85e34 629ec37
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 4 09:35:29 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 4 09:35:29 2015 -0500

----------------------------------------------------------------------
 .../src/main/java/MyProcessor.java                        | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[32/36] incubator-nifi git commit: NIFI-317: - Adding support for exporting the lineage graph as SVG. - Removing support to convert to PNG.

Posted by mc...@apache.org.
NIFI-317:
- Adding support for exporting the lineage graph as SVG.
- Removing support to convert to PNG.

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

Branch: refs/heads/NIFI-250
Commit: 4e85e34c319c847632574c9589b3aa0fdbc355b6
Parents: b0b14ed
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 4 09:05:18 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 4 09:05:18 2015 -0500

----------------------------------------------------------------------
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |  12 --
 .../org/apache/nifi/web/servlet/ConvertSvg.java | 117 -------------------
 .../apache/nifi/web/servlet/DownloadSvg.java    |  93 +++++++++++++++
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |  30 ++++-
 .../js/nf/provenance/nf-provenance-lineage.js   |  15 ++-
 5 files changed, 132 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4e85e34c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 9eaa1b9..f01cbea 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -610,18 +610,6 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>batik</groupId>
-            <artifactId>batik-transcoder</artifactId>
-            <version>1.6-1</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>fop</groupId>
-                    <artifactId>fop</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        
-        <dependency>
             <groupId>javax.servlet</groupId>
             <artifactId>javax.servlet-api</artifactId>
             <scope>provided</scope>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4e85e34c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/ConvertSvg.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/ConvertSvg.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/ConvertSvg.java
deleted file mode 100644
index 6282f41..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/ConvertSvg.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web.servlet;
-
-import java.io.BufferedOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.StringReader;
-import java.net.URLDecoder;
-import javax.servlet.ServletException;
-import javax.servlet.annotation.WebServlet;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import org.apache.batik.dom.svg.SAXSVGDocumentFactory;
-import org.apache.batik.transcoder.TranscoderInput;
-import org.apache.batik.transcoder.TranscoderOutput;
-import org.apache.batik.transcoder.image.PNGTranscoder;
-import org.apache.batik.util.XMLResourceDescriptor;
-import org.apache.commons.io.IOUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.svg.SVGDocument;
-
-/**
- *
- */
-@WebServlet(name = "ConvertSvg", urlPatterns = {"/convert-svg"})
-public class ConvertSvg extends HttpServlet {
-
-    private static final Logger logger = LoggerFactory.getLogger(ConvertSvg.class);
-
-    /**
-     *
-     * @param request servlet request
-     * @param response servlet response
-     * @throws ServletException if a servlet-specific error occurs
-     * @throws IOException if an I/O error occurs
-     */
-    @Override
-    protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
-        final String uri = request.getRequestURL().toString();
-        final String rawSvg = request.getParameter("svg");
-
-        // ensure the image markup has been included
-        if (rawSvg == null) {
-            // set the response status
-            response.setContentType("text/plain");
-            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-
-            // write the response message
-            PrintWriter out = response.getWriter();
-            out.println("SVG must be specified.");
-            return;
-        }
-
-        OutputStream bufferedOut = null;
-        try {
-            // get the svg and decode it, +'s need to be converted
-            final String svg = URLDecoder.decode(rawSvg.replace("+", "%2B"), "UTF-8");
-
-            if (logger.isDebugEnabled()) {
-                logger.debug(svg);
-            }
-
-            String filename = request.getParameter("filename");
-            if (filename == null) {
-                filename = "image.png";
-            } else if (!filename.endsWith(".png")) {
-                filename += ".png";
-            }
-
-            final StringReader reader = new StringReader(svg);
-            final String parser = XMLResourceDescriptor.getXMLParserClassName();
-            final SAXSVGDocumentFactory f = new SAXSVGDocumentFactory(parser);
-            final SVGDocument doc = f.createSVGDocument(uri, reader);
-
-            response.setContentType("image/png");
-            response.setHeader("Content-Disposition", "attachment; filename=" + filename);
-            response.setStatus(HttpServletResponse.SC_OK);
-
-            bufferedOut = new BufferedOutputStream(response.getOutputStream());
-            final TranscoderInput transcoderInput = new TranscoderInput(doc);
-            final TranscoderOutput transcoderOutput = new TranscoderOutput(bufferedOut);
-
-            final PNGTranscoder transcoder = new PNGTranscoder();
-            transcoder.transcode(transcoderInput, transcoderOutput);
-        } catch (final Exception e) {
-            logger.error(e.getMessage(), e);
-
-            // set the response status
-            response.setContentType("text/plain");
-            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-
-            // write the response message
-            PrintWriter out = response.getWriter();
-            out.println("Unable to export image as a PNG.");
-        } finally {
-            IOUtils.closeQuietly(bufferedOut);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4e85e34c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
new file mode 100644
index 0000000..ab76d4e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/java/org/apache/nifi/web/servlet/DownloadSvg.java
@@ -0,0 +1,93 @@
+/*
+ * 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.web.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URLDecoder;
+import javax.servlet.ServletException;
+import javax.servlet.annotation.WebServlet;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+@WebServlet(name = "DownloadSvg", urlPatterns = {"/download-svg"})
+public class DownloadSvg extends HttpServlet {
+
+    private static final Logger logger = LoggerFactory.getLogger(DownloadSvg.class);
+
+    /**
+     *
+     * @param request servlet request
+     * @param response servlet response
+     * @throws ServletException if a servlet-specific error occurs
+     * @throws IOException if an I/O error occurs
+     */
+    @Override
+    protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
+        final String rawSvg = request.getParameter("svg");
+
+        // ensure the image markup has been included
+        if (rawSvg == null) {
+            // set the response status
+            response.setContentType("text/plain");
+            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+
+            // write the response message
+            PrintWriter out = response.getWriter();
+            out.println("SVG must be specified.");
+            return;
+        }
+
+        try {
+            // get the svg and decode it, +'s need to be converted
+            final String svg = URLDecoder.decode(rawSvg.replace("+", "%2B"), "UTF-8");
+
+            if (logger.isDebugEnabled()) {
+                logger.debug(svg);
+            }
+
+            String filename = request.getParameter("filename");
+            if (filename == null) {
+                filename = "image.svg";
+            } else if (!filename.endsWith(".svg")) {
+                filename += ".svg";
+            }
+
+            response.setContentType("image/svg+xml");
+            response.setHeader("Content-Disposition", "attachment; filename=" + filename);
+            response.setStatus(HttpServletResponse.SC_OK);
+
+            response.getWriter().print(svg);
+        } catch (final Exception e) {
+            logger.error(e.getMessage(), e);
+
+            // set the response status
+            response.setContentType("text/plain");
+            response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+
+            // write the response message
+            PrintWriter out = response.getWriter();
+            out.println("Unable to export image as a SVG.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4e85e34c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml
index 8cae6e5..eff7b8d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml
@@ -15,7 +15,9 @@
 -->
 <web-app version="3.0" xmlns="http://java.sun.com/xml/ns/javaee" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd">
     <display-name>nifi</display-name>
+    
     <!-- servlet to map to canvas page -->
+    
     <servlet>
         <servlet-name>NiFiCanvas</servlet-name>
         <jsp-file>/WEB-INF/pages/canvas.jsp</jsp-file>
@@ -24,7 +26,9 @@
         <servlet-name>NiFiCanvas</servlet-name>
         <url-pattern>/canvas</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to summary page -->
+    
     <servlet>
         <servlet-name>NiFiSummary</servlet-name>
         <jsp-file>/WEB-INF/pages/summary.jsp</jsp-file>
@@ -33,7 +37,9 @@
         <servlet-name>NiFiSummary</servlet-name>
         <url-pattern>/summary</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to history page -->
+    
     <servlet>
         <servlet-name>NiFiHistory</servlet-name>
         <jsp-file>/WEB-INF/pages/history.jsp</jsp-file>
@@ -42,7 +48,9 @@
         <servlet-name>NiFiHistory</servlet-name>
         <url-pattern>/history</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to provenance page -->
+    
     <servlet>
         <servlet-name>NiFiProvenance</servlet-name>
         <jsp-file>/WEB-INF/pages/provenance.jsp</jsp-file>
@@ -51,7 +59,9 @@
         <servlet-name>NiFiProvenance</servlet-name>
         <url-pattern>/provenance</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to counters page -->
+    
     <servlet>
         <servlet-name>NiFiCounters</servlet-name>
         <jsp-file>/WEB-INF/pages/counters.jsp</jsp-file>
@@ -60,7 +70,9 @@
         <servlet-name>NiFiCounters</servlet-name>
         <url-pattern>/counters</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to templates page -->
+    
     <servlet>
         <servlet-name>NiFiTemplates</servlet-name>
         <jsp-file>/WEB-INF/pages/templates.jsp</jsp-file>
@@ -69,7 +81,9 @@
         <servlet-name>NiFiTemplates</servlet-name>
         <url-pattern>/templates</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to users page -->
+    
     <servlet>
         <servlet-name>NiFiUsers</servlet-name>
         <jsp-file>/WEB-INF/pages/users.jsp</jsp-file>
@@ -78,7 +92,9 @@
         <servlet-name>NiFiUsers</servlet-name>
         <url-pattern>/users</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to cluster page -->
+    
     <servlet>
         <servlet-name>NiFiCluster</servlet-name>
         <jsp-file>/WEB-INF/pages/cluster.jsp</jsp-file>
@@ -87,7 +103,9 @@
         <servlet-name>NiFiCluster</servlet-name>
         <url-pattern>/cluster</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to map to bulletin board page -->
+    
     <servlet>
         <servlet-name>BulletinBoard</servlet-name>
         <jsp-file>/WEB-INF/pages/bulletin-board.jsp</jsp-file>
@@ -96,15 +114,18 @@
         <servlet-name>BulletinBoard</servlet-name>
         <url-pattern>/bulletin-board</url-pattern>
     </servlet-mapping>
+    
     <!-- servlet to support image downloading -->
+    
     <servlet>
-        <servlet-name>ConvertSvg</servlet-name>
-        <servlet-class>org.apache.nifi.web.servlet.ConvertSvg</servlet-class>
+        <servlet-name>DownloadSvg</servlet-name>
+        <servlet-class>org.apache.nifi.web.servlet.DownloadSvg</servlet-class>
     </servlet>
     <servlet-mapping>
-        <servlet-name>ConvertSvg</servlet-name>
-        <url-pattern>/convert-svg</url-pattern>
+        <servlet-name>DownloadSvg</servlet-name>
+        <url-pattern>/download-svg</url-pattern>
     </servlet-mapping>
+    
     <filter>
         <filter-name>IeEdgeHeader</filter-name>
         <filter-class>org.apache.nifi.web.filter.IeEdgeHeader</filter-class>
@@ -113,6 +134,7 @@
         <filter-name>IeEdgeHeader</filter-name>
         <url-pattern>/*</url-pattern>
     </filter-mapping>
+    
     <welcome-file-list>
         <welcome-file>canvas.jsp</welcome-file>
         <welcome-file>/WEB-INF/pages/canvas.jsp</welcome-file>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4e85e34c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
index 1f05caa..03b08f0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-lineage.js
@@ -531,6 +531,10 @@ nf.ProvenanceLineage = (function () {
         var svg = d3.select('#provenance-lineage-container').append('svg:svg')
                 .attr('width', width)
                 .attr('height', height)
+                .style({
+                    'font-family': 'Verdana, Arial, sans-serif',
+                    'font-size': '10px'
+                })
                 .call(lineageZoom)
                 .on('dblclick.zoom', null)
                 .on('mousedown', function (d) {
@@ -1210,7 +1214,7 @@ nf.ProvenanceLineage = (function () {
         // add the initial lineage
         addLineage(lineageResults.nodes, lineageResults.links);
     };
-
+    
     return {
         /**
          * Initializes the lineage graph.
@@ -1276,7 +1280,14 @@ nf.ProvenanceLineage = (function () {
                     return '<g' + before + 'transform="translate(' + x + ',' + y + ')"' + after + '>';
                 });
 
-                nf.Common.submit('POST', './convert-svg', {
+                // namespaces
+                svg = svg.replace(/^<svg/, '<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" version="1.1"');
+
+                // doctype
+                svg = '<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">\n' + svg;
+
+                // send to server to initiate download... client side only support is too browser specific at this point
+                nf.Common.submit('POST', './download-svg', {
                     'filename': 'provenance',
                     'svg': encodeURIComponent(svg)
                 });


[09/36] incubator-nifi git commit: NIFI-295: - Considering the default value when a property is unset.

Posted by mc...@apache.org.
NIFI-295:
- Considering the default value when a property is unset.

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

Branch: refs/heads/NIFI-250
Commit: 600a47be166e942b64bcc7c624af5db7b5c17cc2
Parents: 94b3903
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 13:40:07 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 13:40:07 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/web/controller/ControllerFacade.java | 15 +++++++++++----
 1 file changed, 11 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/600a47be/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index b2a1ae4..b009581 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -1190,17 +1190,24 @@ public class ControllerFacade implements ControllerServiceProvider {
         for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getProperties().entrySet()) {
             final PropertyDescriptor descriptor = entry.getKey();
             
-            addIfAppropriate(searchStr, descriptor.getName(), "Property", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property", matches);
+            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
+            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
             
             // never include sensitive properties values in search results
             if (descriptor.isSensitive()) {
                 continue;
             }
             
-            final String value = entry.getValue();
+            String value = entry.getValue();
+            
+            // if unset consider default value
+            if (value == null) {
+                value = descriptor.getDefaultValue();
+            }
+            
+            // evaluate if the value matches the search criteria
             if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property: " + descriptor.getName() + " - " + value);
+                matches.add("Property value: " + descriptor.getName() + " - " + value);
             }
         }
 


[26/36] incubator-nifi git commit: NIFI-268: Fixed unit tests

Posted by mc...@apache.org.
NIFI-268: Fixed unit tests


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

Branch: refs/heads/NIFI-250
Commit: 7724dadf0d4a268181a59c649b3600186e892bdd
Parents: 3b399d7
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 3 12:20:09 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 3 12:20:09 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/util/TestStandardProcessorTestRunner.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7724dadf/nifi/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java b/nifi/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
index 37bcf23..a561982 100644
--- a/nifi/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
+++ b/nifi/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
@@ -20,8 +20,10 @@ 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 org.junit.Ignore;
 import org.junit.Test;
 
+@Ignore("This should not be enabled until we actually fail processor unit tests for using deprecated methods, which should happen in 0.1.0")
 public class TestStandardProcessorTestRunner {
 
     @Test(expected=AssertionError.class)


[24/36] incubator-nifi git commit: NIFI-310: Added unit tests for PostHTTP

Posted by mc...@apache.org.
NIFI-310: Added unit tests for PostHTTP


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

Branch: refs/heads/NIFI-250
Commit: 10714efbd3d73b1abb3d8f5e48114df05ab57b96
Parents: 102e3cb
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 3 11:57:58 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 3 11:57:58 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/CaptureServlet.java     |  54 ++++
 .../nifi/processors/standard/TestPostHTTP.java  | 249 +++++++++++++++++++
 .../nifi/processors/standard/TestServer.java    |  27 +-
 3 files changed, 323 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/10714efb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/CaptureServlet.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/CaptureServlet.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/CaptureServlet.java
new file mode 100644
index 0000000..cf4f609
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/CaptureServlet.java
@@ -0,0 +1,54 @@
+/*
+ * 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.standard;
+
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.activemq.util.ByteArrayOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class CaptureServlet extends HttpServlet {
+    private static final long serialVersionUID = 8402271018449653919L;
+
+    private volatile byte[] lastPost;
+    
+    public byte[] getLastPost() {
+        return lastPost;
+    }
+    
+    @Override
+    protected void doPost(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        StreamUtils.copy(request.getInputStream(), baos);
+        this.lastPost = baos.toByteArray();
+        
+        response.setStatus(Status.OK.getStatusCode());
+    }
+    
+    @Override
+    protected void doHead(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
+        response.setHeader("Accept", "application/flowfile-v3,application/flowfile-v2");
+        response.setHeader("x-nifi-transfer-protocol-version", "1");
+        response.setHeader("Accept-Encoding", "gzip");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/10714efb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
new file mode 100644
index 0000000..dcda136
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
@@ -0,0 +1,249 @@
+/*
+ * 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.standard;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.ssl.StandardSSLContextService;
+import org.apache.nifi.util.FlowFileUnpackagerV3;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.junit.After;
+import org.junit.Test;
+
+public class TestPostHTTP {
+    private TestServer server;
+    private TestRunner runner;
+    private CaptureServlet servlet;
+    
+    private void setup(final Map<String, String> sslProperties) throws Exception {
+        // set up web service
+        ServletHandler handler = new ServletHandler();
+        handler.addServletWithMapping(CaptureServlet.class, "/*");
+        servlet = (CaptureServlet) handler.getServlets()[0].getServlet();
+        
+        // create the service
+        server = new TestServer(sslProperties);
+        server.addHandler(handler);
+        server.startServer();
+        
+        runner = TestRunners.newTestRunner(PostHTTP.class);
+    }
+    
+    @After
+    public void cleanup() throws Exception {
+        if (server != null) {
+            server.shutdownServer();
+            server = null;
+        }
+    }
+    
+    @Test
+    public void testTruststoreSSLOnly() throws Exception {
+        final Map<String, String> sslProps = new HashMap<>();
+        sslProps.put(TestServer.NEED_CLIENT_AUTH, "false");
+        sslProps.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        sslProps.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        setup(sslProps);
+        
+        final SSLContextService sslContextService = new StandardSSLContextService();
+        runner.addControllerService("ssl-context", sslContextService);
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
+        runner.enableControllerService(sslContextService);
+        
+        runner.setProperty(PostHTTP.URL, server.getSecureUrl());
+        runner.setProperty(PostHTTP.SSL_CONTEXT_SERVICE, "ssl-context");
+        
+        runner.enqueue("Hello world".getBytes());
+        runner.run();
+        
+        runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS, 1);
+    }
+    
+    
+    @Test
+    public void testTwoWaySSL() throws Exception {
+        final Map<String, String> sslProps = new HashMap<>();
+        sslProps.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        sslProps.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        sslProps.put(TestServer.NEED_CLIENT_AUTH, "true");
+        setup(sslProps);
+        
+        final SSLContextService sslContextService = new StandardSSLContextService();
+        runner.addControllerService("ssl-context", sslContextService);
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
+        runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
+        runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
+        runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE_TYPE, "JKS");
+        runner.enableControllerService(sslContextService);
+        
+        runner.setProperty(PostHTTP.URL, server.getSecureUrl());
+        runner.setProperty(PostHTTP.SSL_CONTEXT_SERVICE, "ssl-context");
+        
+        runner.enqueue("Hello world".getBytes());
+        runner.run();
+        
+        runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS, 1);
+    }
+    
+    @Test
+    public void testOneWaySSLWhenServerConfiguredForTwoWay() throws Exception {
+        final Map<String, String> sslProps = new HashMap<>();
+        sslProps.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        sslProps.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        sslProps.put(TestServer.NEED_CLIENT_AUTH, "true");
+        setup(sslProps);
+        
+        final SSLContextService sslContextService = new StandardSSLContextService();
+        runner.addControllerService("ssl-context", sslContextService);
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
+        runner.enableControllerService(sslContextService);
+        
+        runner.setProperty(PostHTTP.URL, server.getSecureUrl());
+        runner.setProperty(PostHTTP.SSL_CONTEXT_SERVICE, "ssl-context");
+        
+        runner.enqueue("Hello world".getBytes());
+        runner.run();
+        
+        runner.assertAllFlowFilesTransferred(PostHTTP.REL_FAILURE, 1);
+    }
+    
+    @Test
+    public void testSendAsFlowFile() throws Exception {
+        setup(null);
+        runner.setProperty(PostHTTP.URL, server.getUrl());
+        runner.setProperty(PostHTTP.SEND_AS_FLOWFILE, "true");
+
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("abc", "cba");
+        
+        runner.enqueue("Hello".getBytes(), attrs);
+        attrs.put("abc", "abc");
+        attrs.put("filename", "xyz.txt");
+        runner.enqueue("World".getBytes(), attrs);
+        
+        runner.run(1);
+        runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS);
+        
+        final byte[] lastPost = servlet.getLastPost();
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        final ByteArrayInputStream bais = new ByteArrayInputStream(lastPost);
+        
+        FlowFileUnpackagerV3 unpacker = new FlowFileUnpackagerV3();
+        
+        // unpack first flowfile received
+        Map<String, String> receivedAttrs = unpacker.unpackageFlowFile(bais, baos);
+        byte[] contentReceived = baos.toByteArray();
+        assertEquals("Hello", new String(contentReceived));
+        assertEquals("cba", receivedAttrs.get("abc"));
+        
+        assertTrue( unpacker.hasMoreData() );
+        
+        baos.reset();
+        receivedAttrs = unpacker.unpackageFlowFile(bais, baos);
+        contentReceived = baos.toByteArray();
+        
+        assertEquals("World", new String(contentReceived));
+        assertEquals("abc", receivedAttrs.get("abc"));
+        assertEquals("xyz.txt", receivedAttrs.get("filename"));
+    }
+    
+    
+    @Test
+    public void testSendAsFlowFileSecure() throws Exception {
+        final Map<String, String> sslProps = new HashMap<>();
+        sslProps.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        sslProps.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+        sslProps.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        sslProps.put(TestServer.NEED_CLIENT_AUTH, "true");
+        setup(sslProps);
+        
+        final SSLContextService sslContextService = new StandardSSLContextService();
+        runner.addControllerService("ssl-context", sslContextService);
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
+        runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
+        runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
+        runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
+        runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE_TYPE, "JKS");
+        runner.enableControllerService(sslContextService);
+        
+        runner.setProperty(PostHTTP.URL, server.getSecureUrl());
+        runner.setProperty(PostHTTP.SEND_AS_FLOWFILE, "true");
+        runner.setProperty(PostHTTP.SSL_CONTEXT_SERVICE, "ssl-context");
+
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("abc", "cba");
+        
+        runner.enqueue("Hello".getBytes(), attrs);
+        attrs.put("abc", "abc");
+        attrs.put("filename", "xyz.txt");
+        runner.enqueue("World".getBytes(), attrs);
+        
+        runner.run(1);
+        runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS);
+        
+        final byte[] lastPost = servlet.getLastPost();
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        final ByteArrayInputStream bais = new ByteArrayInputStream(lastPost);
+        
+        FlowFileUnpackagerV3 unpacker = new FlowFileUnpackagerV3();
+        
+        // unpack first flowfile received
+        Map<String, String> receivedAttrs = unpacker.unpackageFlowFile(bais, baos);
+        byte[] contentReceived = baos.toByteArray();
+        assertEquals("Hello", new String(contentReceived));
+        assertEquals("cba", receivedAttrs.get("abc"));
+        
+        assertTrue( unpacker.hasMoreData() );
+        
+        baos.reset();
+        receivedAttrs = unpacker.unpackageFlowFile(bais, baos);
+        contentReceived = baos.toByteArray();
+        
+        assertEquals("World", new String(contentReceived));
+        assertEquals("abc", receivedAttrs.get("abc"));
+        assertEquals("xyz.txt", receivedAttrs.get("filename"));
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/10714efb/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
index 53b79e6..abdff6e 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
@@ -28,6 +28,7 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
  * Test server to assist with unit tests that requires a server to be stood up.
  */
 public class TestServer {
+    public static final String NEED_CLIENT_AUTH = "clientAuth";
     
     private Server jetty;
     private boolean secure = false;
@@ -85,13 +86,25 @@ public class TestServer {
      */
     private void createSecureConnector(final Map<String, String> sslProperties) {
         SslContextFactory ssl = new SslContextFactory();
-        ssl.setKeyStorePath(sslProperties.get(StandardSSLContextService.KEYSTORE.getName()));
-        ssl.setKeyStorePassword(sslProperties.get(StandardSSLContextService.KEYSTORE_PASSWORD.getName()));
-        ssl.setKeyStoreType(sslProperties.get(StandardSSLContextService.KEYSTORE_TYPE.getName()));
-        ssl.setTrustStorePath(sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()));
-        ssl.setTrustStorePassword(sslProperties.get(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName()));
-        ssl.setTrustStoreType(sslProperties.get(StandardSSLContextService.TRUSTSTORE_TYPE.getName()));
-        ssl.setNeedClientAuth(true);
+        
+        if ( sslProperties.get(StandardSSLContextService.KEYSTORE.getName()) != null ) {
+            ssl.setKeyStorePath(sslProperties.get(StandardSSLContextService.KEYSTORE.getName()));
+            ssl.setKeyStorePassword(sslProperties.get(StandardSSLContextService.KEYSTORE_PASSWORD.getName()));
+            ssl.setKeyStoreType(sslProperties.get(StandardSSLContextService.KEYSTORE_TYPE.getName()));
+        }
+        
+        if ( sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()) != null ) {
+            ssl.setTrustStorePath(sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()));
+            ssl.setTrustStorePassword(sslProperties.get(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName()));
+            ssl.setTrustStoreType(sslProperties.get(StandardSSLContextService.TRUSTSTORE_TYPE.getName()));
+        }
+        
+        final String clientAuth = sslProperties.get(NEED_CLIENT_AUTH);
+        if ( clientAuth == null ) {
+            ssl.setNeedClientAuth(true);
+        } else {
+            ssl.setNeedClientAuth(Boolean.parseBoolean(clientAuth));
+        }
 
         // build the connector
         final ServerConnector https = new ServerConnector(jetty, ssl);


[21/36] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: 4b852ebc651e2af7e9ba8b72494b661a86400993
Parents: d145e20 a738be8
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Feb 3 09:51:59 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Feb 3 09:51:59 2015 -0500

----------------------------------------------------------------------
 README.md                       | 14 ++++++++++++++
 nifi-nar-maven-plugin/README.md | 14 ++++++++++++++
 nifi-nar-maven-plugin/pom.xml   |  8 ++++----
 nifi-site/README.md             | 14 ++++++++++++++
 nifi/README.md                  | 14 ++++++++++++++
 nifi/nifi-assembly/pom.xml      |  1 +
 nifi/pom.xml                    | 21 +++++++++------------
 7 files changed, 70 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[10/36] incubator-nifi git commit: NIFI-305: Slight refactorings to provide more flexibility in concrete implementations

Posted by mc...@apache.org.
NIFI-305: Slight refactorings to provide more flexibility in concrete implementations


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

Branch: refs/heads/NIFI-250
Commit: ec7f7e7717750276a6cbf56edbfcca49b1299fa5
Parents: c01dff5
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 2 13:51:51 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 2 13:51:51 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/BinFiles.java      | 72 +++-----------------
 .../nifi/processors/standard/MergeContent.java  | 69 ++++++++++---------
 2 files changed, 46 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ec7f7e77/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
index 0a65c59..3d7dba1 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
@@ -19,11 +19,8 @@ package org.apache.nifi.processors.standard;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Queue;
-import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
@@ -39,7 +36,6 @@ import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessSessionFactory;
-import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -99,38 +95,9 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
     public static final Relationship REL_ORIGINAL = new Relationship.Builder().name("original").description("The FlowFiles that were used to create the bundle").build();
     public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("If the bundle cannot be created, all FlowFiles that would have been used to created the bundle will be transferred to failure").build();
 
-    private Set<Relationship> relationships;
-    private List<PropertyDescriptor> descriptors;
     private final BinManager binManager = new BinManager();
-
     private final Queue<Bin> readyBins = new LinkedBlockingQueue<>();
 
-    @Override
-    protected final void init(final ProcessorInitializationContext context) {
-
-    	final Set<Relationship> relationships = new HashSet<>();
-        relationships.add(REL_ORIGINAL);
-        relationships.add(REL_FAILURE);
-        Set<Relationship> additionalRelationships = defineAdditionalRelationships();
-        if (additionalRelationships != null) {
-        	relationships.addAll(additionalRelationships);
-        }
-        this.relationships = Collections.unmodifiableSet(relationships);
-
-        final List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.add(MIN_ENTRIES);
-        descriptors.add(MAX_ENTRIES);
-        descriptors.add(MIN_SIZE);
-        descriptors.add(MAX_SIZE);
-        descriptors.add(MAX_BIN_AGE);
-        descriptors.add(MAX_BIN_COUNT);
-        List<PropertyDescriptor> additionalPropertyDescriptors = this.defineAdditionalPropertyDescriptors();
-        if (additionalPropertyDescriptors != null) {
-        	descriptors.addAll(additionalPropertyDescriptors);
-        }
-
-        this.descriptors = Collections.unmodifiableList(descriptors);
-    }
 
     @OnStopped
     public final void resetState() {
@@ -144,27 +111,6 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
         }
     }
 
-    @Override
-    public final Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    protected final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return descriptors;
-    }
-
-    /**
-     * Allows any additional relationships to be defined.
-     * @return Relationships to be added in the init() method
-     */ 
-    protected abstract Set<Relationship> defineAdditionalRelationships();
-    
-    /**
-     * Allows any additional property descriptors to be defined.
-     * @return Properties to be added in the init() method
-     */
-    protected abstract List<PropertyDescriptor> defineAdditionalPropertyDescriptors();
 
 	/**
 	 * Allows general pre-processing of a flow file before it is offered to a
@@ -213,14 +159,14 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
 	 * to Failure and commit their sessions.  If false, the 
 	 * processBins() method will transfer the files to Original and commit
 	 * the sessions
-	 * @throws Exception
-	 *             This will be handled appropriately, and all flow files in the
-	 *             bin will be transferred to failure and the session rolled
-	 *             back
+	 * 
+	 * @throws ProcessException if any problem arises while processing a bin
+	 *             of FlowFiles. All flow files in the
+	 *             bin will be transferred to failure and the ProcessSession provided by
+	 *             the 'session' argument rolled back
 	 */
-	protected abstract boolean processBin(Bin unmodifiableBin,
-			List<FlowFileSessionWrapper> binContents, ProcessContext context,
-			ProcessSession session, ProcessorLog logger) throws Exception;
+	protected abstract boolean processBin(Bin unmodifiableBin, 
+	        List<FlowFileSessionWrapper> binContents, ProcessContext context, ProcessSession session) throws ProcessException;
 
     /**
 	 * Allows additional custom validation to be done. This will be called from
@@ -288,8 +234,8 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
 
     	boolean binAlreadyCommitted = false;
         try {
-        	binAlreadyCommitted = this.processBin(bin, binCopy, context, session, logger);
-        } catch (final Exception e) {
+        	binAlreadyCommitted = this.processBin(bin, binCopy, context, session);
+        } catch (final ProcessException e) {
             logger.error("Failed to process bundle of {} files due to {}", new Object[]{binCopy.size(), e});
 
             for (final FlowFileSessionWrapper wrapper : binCopy) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ec7f7e77/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index 73cb5a6..a78bc07 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -46,10 +46,10 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -198,56 +198,62 @@ public class MergeContent extends BinFiles {
 
     public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+");
 
+
 	@Override
-	protected Set<Relationship> defineAdditionalRelationships() {
-        final Set<Relationship> relationships = new HashSet<>();
+	public Set<Relationship> getRelationships() {
+	    final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
         relationships.add(REL_MERGED);
-        
         return relationships;
-    }
-
+	}
+	
+	
 	@Override
-	protected List<PropertyDescriptor> defineAdditionalPropertyDescriptors() {
-		final List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.add(MERGE_STRATEGY);
-        descriptors.add(MERGE_FORMAT);
-        descriptors.add(ATTRIBUTE_STRATEGY);
-        descriptors.add(CORRELATION_ATTRIBUTE_NAME);
+	protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+	    final List<PropertyDescriptor> descriptors = new ArrayList<>();
+	    descriptors.add(MERGE_STRATEGY);
+	    descriptors.add(MERGE_FORMAT);
+	    descriptors.add(ATTRIBUTE_STRATEGY);
+	    descriptors.add(CORRELATION_ATTRIBUTE_NAME);
+        descriptors.add(MIN_ENTRIES);
+        descriptors.add(MAX_ENTRIES);
+        descriptors.add(MIN_SIZE);
+        descriptors.add(MAX_SIZE);
+        descriptors.add(MAX_BIN_AGE);
+        descriptors.add(MAX_BIN_COUNT);
         descriptors.add(HEADER);
         descriptors.add(FOOTER);
         descriptors.add(DEMARCATOR);
         descriptors.add(COMPRESSION_LEVEL);
         descriptors.add(KEEP_PATH);
-        
         return descriptors;
 	}
-
+	
     private byte[] readContent(final String filename) throws IOException {
         return Files.readAllBytes(Paths.get(filename));
     }
 
 
 	@Override
-	protected FlowFile preprocessFlowFile(ProcessContext context,
-			ProcessSession session, FlowFile flowFile) {
-		
+	protected FlowFile preprocessFlowFile(final ProcessContext context, final ProcessSession session, final FlowFile flowFile) {
+	    FlowFile processed = flowFile;
         // handle backward compatibility with old segment attributes
-        if (flowFile.getAttribute(FRAGMENT_COUNT_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_COUNT_ATTRIBUTE) != null) {
-            flowFile = session.putAttribute(flowFile, FRAGMENT_COUNT_ATTRIBUTE, flowFile.getAttribute(SEGMENT_COUNT_ATTRIBUTE));
+        if (processed.getAttribute(FRAGMENT_COUNT_ATTRIBUTE) == null && processed.getAttribute(SEGMENT_COUNT_ATTRIBUTE) != null) {
+            processed = session.putAttribute(processed, FRAGMENT_COUNT_ATTRIBUTE, processed.getAttribute(SEGMENT_COUNT_ATTRIBUTE));
         }
-        if (flowFile.getAttribute(FRAGMENT_INDEX_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_INDEX_ATTRIBUTE) != null) {
-            flowFile = session.putAttribute(flowFile, FRAGMENT_INDEX_ATTRIBUTE, flowFile.getAttribute(SEGMENT_INDEX_ATTRIBUTE));
+        if (processed.getAttribute(FRAGMENT_INDEX_ATTRIBUTE) == null && processed.getAttribute(SEGMENT_INDEX_ATTRIBUTE) != null) {
+            processed = session.putAttribute(processed, FRAGMENT_INDEX_ATTRIBUTE, processed.getAttribute(SEGMENT_INDEX_ATTRIBUTE));
         }
-        if (flowFile.getAttribute(FRAGMENT_ID_ATTRIBUTE) == null && flowFile.getAttribute(SEGMENT_ID_ATTRIBUTE) != null) {
-            flowFile = session.putAttribute(flowFile, FRAGMENT_ID_ATTRIBUTE, flowFile.getAttribute(SEGMENT_ID_ATTRIBUTE));
+        if (processed.getAttribute(FRAGMENT_ID_ATTRIBUTE) == null && processed.getAttribute(SEGMENT_ID_ATTRIBUTE) != null) {
+            processed = session.putAttribute(processed, FRAGMENT_ID_ATTRIBUTE, processed.getAttribute(SEGMENT_ID_ATTRIBUTE));
         }
         
-        return flowFile;
+        return processed;
 	}
 
 	@Override
-	protected String getGroupId(ProcessContext context, FlowFile flowFile) {
-
+	protected String getGroupId(final ProcessContext context, final FlowFile flowFile) {
         final String correlationAttributeName = context.getProperty(CORRELATION_ATTRIBUTE_NAME).getValue();
         String groupId = (correlationAttributeName == null) ? null : flowFile.getAttribute(correlationAttributeName);
 
@@ -260,16 +266,15 @@ public class MergeContent extends BinFiles {
 	}
 
 	@Override
-	protected void setUpBinManager(BinManager binManager, ProcessContext context) {
+	protected void setUpBinManager(final BinManager binManager, final ProcessContext context) {
 		if (MERGE_STRATEGY_DEFRAGMENT.equals(context.getProperty(MERGE_STRATEGY).getValue())) {
             binManager.setFileCountAttribute(FRAGMENT_COUNT_ATTRIBUTE);
         }
 	}
 
 	@Override
-	protected boolean processBin(Bin unmodifiableBin,
-			List<FlowFileSessionWrapper> binCopy, ProcessContext context,
-			ProcessSession session, ProcessorLog logger) throws Exception {
+	protected boolean processBin(final Bin unmodifiableBin, final List<FlowFileSessionWrapper> binCopy, final ProcessContext context,
+			final ProcessSession session) throws ProcessException {
 
         final String mergeFormat = context.getProperty(MERGE_FORMAT).getValue();
         MergeBin merger;
@@ -314,7 +319,7 @@ public class MergeContent extends BinFiles {
             // Fail the flow files and commit them
             if (error != null) {
                 final String binDescription = binCopy.size() <= 10 ? binCopy.toString() : binCopy.size() + " FlowFiles";
-                logger.error(error + "; routing {} to failure", new Object[]{binDescription});
+                getLogger().error(error + "; routing {} to failure", new Object[]{binDescription});
                 for ( final FlowFileSessionWrapper wrapper : binCopy ) {
                     wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
                     wrapper.getSession().commit();
@@ -341,7 +346,7 @@ public class MergeContent extends BinFiles {
         bundle = session.putAllAttributes(bundle, bundleAttributes);
 
         final String inputDescription = (binCopy.size() < 10) ? binCopy.toString() : binCopy.size() + " FlowFiles";
-        logger.info("Merged {} into {}", new Object[]{inputDescription, bundle});
+        getLogger().info("Merged {} into {}", new Object[]{inputDescription, bundle});
         session.transfer(bundle, REL_MERGED);
 
         // We haven't committed anything, parent will take care of it


[06/36] incubator-nifi git commit: NIFI-307: - Updating how action buttons are invoked in tables throughout the application. NIFI-311: - Addressing visibility and behavior issue with the refresh button in the component cluster summary dialog.

Posted by mc...@apache.org.
NIFI-307:
- Updating how action buttons are invoked in tables throughout the application.
NIFI-311:
- Addressing visibility and behavior issue with the refresh button in the component cluster summary dialog.

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

Branch: refs/heads/NIFI-250
Commit: b4d07e262f05ce16b4731d9a4b57de1d8d35848e
Parents: 9cd9d12
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 12:45:53 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 12:45:53 2015 -0500

----------------------------------------------------------------------
 .../webapp/js/nf/summary/nf-summary-table.js    | 523 +++++++------------
 .../src/main/webapp/js/nf/summary/nf-summary.js |   6 -
 2 files changed, 190 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b4d07e26/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js
index 969d7bf..a4583bc 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js
@@ -263,7 +263,7 @@ nf.SummaryTable = (function () {
 
         // define a custom formatter for showing more processor details
         var moreProcessorDetails = function (row, cell, value, columnDef, dataContext) {
-            var markup = '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 5px; float: left;" onclick="javascript:nf.SummaryTable.showProcessorDetails(\'' + row + '\');"/>';
+            var markup = '<img src="images/iconDetails.png" title="View Details" class="pointer show-processor-details" style="margin-top: 5px; float: left;"/>';
 
             // if there are bulletins, render them on the graph
             if (!nf.Common.isEmpty(dataContext.bulletins)) {
@@ -333,26 +333,26 @@ nf.SummaryTable = (function () {
                 var markup = '';
 
                 if (isInShell) {
-                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.goToProcessor(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer go-to" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 if (nf.Common.SUPPORTS_SVG) {
                     if (isClustered) {
-                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterProcessorStatusHistory(\'' + row + '\');"/>&nbsp;';
+                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer show-cluster-processor-status-history" style="margin-top: 2px;"/>&nbsp;';
                     } else {
-                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showProcessorStatusHistory(\'' + row + '\');"/>&nbsp;';
+                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer show-processor-status-history" style="margin-top: 2px;"/>&nbsp;';
                     }
                 }
 
                 if (isClustered) {
-                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterProcessorSummary(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer show-cluster-processor-summary" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 return markup;
             };
 
             // define the action column for clusters and within the shell
-            processorsColumnModel.push({id: 'action', name: '&nbsp;', formatter: processorActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
+            processorsColumnModel.push({id: 'actions', name: '&nbsp;', formatter: processorActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
         }
 
         // initialize the templates table
@@ -394,6 +394,38 @@ nf.SummaryTable = (function () {
             }, processorsData);
         });
 
+        // configure a click listener
+        processorsGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = processorsData.getItem(args.row);
+
+            // determine the desired action
+            if (processorsGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('go-to')) {
+                    goTo(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-processor-status-history')) {
+                    nf.StatusHistory.showClusterProcessorChart(item.groupId, item.id);
+                } else if (target.hasClass('show-processor-status-history')) {
+                    nf.StatusHistory.showStandaloneProcessorChart(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-processor-summary')) {
+                    // load the cluster processor summary
+                    loadClusterProcessorSummary(item.id);
+
+                    // hide the summary loading indicator
+                    $('#summary-loading-container').hide();
+
+                    // show the dialog
+                    $('#cluster-processor-summary-dialog').modal('show');
+                }
+            } else if (processorsGrid.getColumns()[args.cell].id === 'moreDetails') {
+                if (target.hasClass('show-processor-details')) {
+                    nf.ProcessorDetails.showDetails(item.groupId, item.id);
+                }
+            }
+        });
+
         // wire up the dataview to the grid
         processorsData.onRowCountChanged.subscribe(function (e, args) {
             processorsGrid.updateRowCount();
@@ -464,6 +496,11 @@ nf.SummaryTable = (function () {
                 }
             }
         });
+        
+        // cluster processor refresh
+        nf.Common.addHoverEffect('#cluster-processor-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
+            loadClusterProcessorSummary($('#cluster-processor-id').text());
+        });
 
         // initialize the cluster processor column model
         var clusterProcessorsColumnModel = [
@@ -524,7 +561,7 @@ nf.SummaryTable = (function () {
 
         // define a custom formatter for showing more processor details
         var moreConnectionDetails = function (row, cell, value, columnDef, dataContext) {
-            return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 5px;" onclick="javascript:nf.SummaryTable.showConnectionDetails(\'' + row + '\');"/>';
+            return '<img src="images/iconDetails.png" title="View Details" class="pointer show-connection-details" style="margin-top: 5px;"/>';
         };
 
         // define the input, read, written, and output columns (reused between both tables)
@@ -548,26 +585,26 @@ nf.SummaryTable = (function () {
                 var markup = '';
 
                 if (isInShell) {
-                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.goToConnection(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer go-to" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 if (nf.Common.SUPPORTS_SVG) {
                     if (isClustered) {
-                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterConnectionStatusHistory(\'' + row + '\');"/>&nbsp;';
+                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer show-cluster-connection-status-history" style="margin-top: 2px;"/>&nbsp;';
                     } else {
-                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showConnectionStatusHistory(\'' + row + '\');"/>&nbsp;';
+                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer show-connection-status-history" style="margin-top: 2px;"/>&nbsp;';
                     }
                 }
 
                 if (isClustered) {
-                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterConnectionSummary(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer show-cluster-connection-summary" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 return markup;
             };
 
             // define the action column for clusters and within the shell
-            connectionsColumnModel.push({id: 'action', name: '&nbsp;', formatter: connectionActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
+            connectionsColumnModel.push({id: 'actions', name: '&nbsp;', formatter: connectionActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
         }
 
         // initialize the templates table
@@ -609,6 +646,38 @@ nf.SummaryTable = (function () {
             }, connectionsData);
         });
 
+        // configure a click listener
+        connectionsGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = connectionsData.getItem(args.row);
+
+            // determine the desired action
+            if (connectionsGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('go-to')) {
+                    goTo(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-connection-status-history')) {
+                    nf.StatusHistory.showClusterConnectionChart(item.groupId, item.id);
+                } else if (target.hasClass('show-connection-status-history')) {
+                    nf.StatusHistory.showStandaloneConnectionChart(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-connection-summary')) {
+                    // load the cluster processor summary
+                    loadClusterConnectionSummary(item.id);
+
+                    // hide the summary loading indicator
+                    $('#summary-loading-container').hide();
+
+                    // show the dialog
+                    $('#cluster-connection-summary-dialog').modal('show');
+                }
+            } else if (connectionsGrid.getColumns()[args.cell].id === 'moreDetails') {
+                if (target.hasClass('show-connection-details')) {
+                    nf.ConnectionDetails.showDetails(item.groupId, item.id);
+                }
+            }
+        });
+
         // wire up the dataview to the grid
         connectionsData.onRowCountChanged.subscribe(function (e, args) {
             connectionsGrid.updateRowCount();
@@ -651,6 +720,11 @@ nf.SummaryTable = (function () {
                 }
             }
         });
+        
+        // cluster connection refresh
+        nf.Common.addHoverEffect('#cluster-connection-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
+            loadClusterConnectionSummary($('#cluster-connection-id').text());
+        });
 
         // initialize the cluster processor column model
         var clusterConnectionsColumnModel = [
@@ -734,18 +808,18 @@ nf.SummaryTable = (function () {
                 var markup = '';
 
                 if (isInShell) {
-                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.goToInputPort(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer go-to" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 if (isClustered) {
-                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterInputPortSummary(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer show-cluster-input-port-summary" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 return markup;
             };
 
             // define the action column for clusters and within the shell
-            inputPortsColumnModel.push({id: 'action', name: '&nbsp;', formatter: inputPortActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
+            inputPortsColumnModel.push({id: 'actions', name: '&nbsp;', formatter: inputPortActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
         }
 
         // initialize the input ports table
@@ -787,6 +861,30 @@ nf.SummaryTable = (function () {
             }, inputPortsData);
         });
 
+        // configure a click listener
+        inputPortsGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = inputPortsData.getItem(args.row);
+
+            // determine the desired action
+            if (inputPortsGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('go-to')) {
+                    goTo(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-input-port-summary')) {
+                    // load the cluster processor summary
+                    loadClusterInputPortSummary(item.id);
+
+                    // hide the summary loading indicator
+                    $('#summary-loading-container').hide();
+
+                    // show the dialog
+                    $('#cluster-input-port-summary-dialog').modal('show');
+                }
+            }
+        });
+
         // wire up the dataview to the grid
         inputPortsData.onRowCountChanged.subscribe(function (e, args) {
             inputPortsGrid.updateRowCount();
@@ -857,6 +955,11 @@ nf.SummaryTable = (function () {
                 }
             }
         });
+        
+        // cluster input port refresh
+        nf.Common.addHoverEffect('#cluster-input-port-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
+            loadClusterInputPortSummary($('#cluster-input-port-id').text());
+        });
 
         // initialize the cluster input port column model
         var clusterInputPortsColumnModel = [
@@ -927,18 +1030,18 @@ nf.SummaryTable = (function () {
                 var markup = '';
 
                 if (isInShell) {
-                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.goToOutputPort(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer go-to" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 if (isClustered) {
-                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterOutputPortSummary(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer show-cluster-output-port-summary" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 return markup;
             };
 
             // define the action column for clusters and within the shell
-            outputPortsColumnModel.push({id: 'action', name: '&nbsp;', formatter: outputPortActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
+            outputPortsColumnModel.push({id: 'actions', name: '&nbsp;', formatter: outputPortActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
         }
 
         // initialize the input ports table
@@ -980,6 +1083,30 @@ nf.SummaryTable = (function () {
             }, outputPortsData);
         });
 
+        // configure a click listener
+        outputPortsGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = outputPortsData.getItem(args.row);
+
+            // determine the desired action
+            if (outputPortsGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('go-to')) {
+                    goTo(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-output-port-summary')) {
+                    // load the cluster processor summary
+                    loadClusterOutputPortSummary(item.id);
+
+                    // hide the summary loading indicator
+                    $('#summary-loading-container').hide();
+
+                    // show the dialog
+                    $('#cluster-output-port-summary-dialog').modal('show');
+                }
+            }
+        });
+
         // wire up the dataview to the grid
         outputPortsData.onRowCountChanged.subscribe(function (e, args) {
             outputPortsGrid.updateRowCount();
@@ -1050,6 +1177,11 @@ nf.SummaryTable = (function () {
                 }
             }
         });
+        
+        // cluster output port refresh
+        nf.Common.addHoverEffect('#cluster-output-port-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
+            loadClusterOutputPortSummary($('#cluster-output-port-id').text());
+        });
 
         // initialize the cluster output port column model
         var clusterOutputPortsColumnModel = [
@@ -1152,26 +1284,26 @@ nf.SummaryTable = (function () {
                 var markup = '';
 
                 if (isInShell) {
-                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.goToRemoteProcessGroup(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconGoTo.png" title="Go To" class="pointer go-to" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 if (nf.Common.SUPPORTS_SVG) {
                     if (isClustered) {
-                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterRemoteProcessGroupStatusHistory(\'' + row + '\');"/>&nbsp;';
+                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer show-cluster-remote-process-group-status-history" style="margin-top: 2px;"/>&nbsp;';
                     } else {
-                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showRemoteProcessGroupStatusHistory(\'' + row + '\');"/>&nbsp;';
+                        markup += '<img src="images/iconChart.png" title="Show History" class="pointer show-remote-process-group-status-history" style="margin-top: 2px;"/>&nbsp;';
                     }
                 }
 
                 if (isClustered) {
-                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer" style="margin-top: 2px;" onclick="javascript:nf.SummaryTable.showClusterRemoteProcessGroupSummary(\'' + row + '\');"/>&nbsp;';
+                    markup += '<img src="images/iconClusterSmall.png" title="Show Details" class="pointer show-cluster-remote-process-group-summary" style="margin-top: 2px;"/>&nbsp;';
                 }
 
                 return markup;
             };
 
             // define the action column for clusters and within the shell
-            remoteProcessGroupsColumnModel.push({id: 'action', name: '&nbsp;', formatter: remoteProcessGroupActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
+            remoteProcessGroupsColumnModel.push({id: 'actions', name: '&nbsp;', formatter: remoteProcessGroupActionFormatter, resizable: false, sortable: false, width: 75, maxWidth: 75});
         }
 
         // initialize the remote process groups table
@@ -1213,6 +1345,34 @@ nf.SummaryTable = (function () {
             }, remoteProcessGroupsData);
         });
 
+        // configure a click listener
+        remoteProcessGroupsGrid.onClick.subscribe(function (e, args) {
+            var target = $(e.target);
+
+            // get the node at this row
+            var item = remoteProcessGroupsData.getItem(args.row);
+
+            // determine the desired action
+            if (remoteProcessGroupsGrid.getColumns()[args.cell].id === 'actions') {
+                if (target.hasClass('go-to')) {
+                    goTo(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-remote-process-group-status-history')) {
+                    nf.StatusHistory.showClusterRemoteProcessGroupChart(item.groupId, item.id);
+                } else if (target.hasClass('show-remote-process-group-status-history')) {
+                    nf.StatusHistory.showStandaloneRemoteProcessGroupChart(item.groupId, item.id);
+                } else if (target.hasClass('show-cluster-remote-process-group-summary')) {
+                    // load the cluster processor summary
+                    loadClusterRemoteProcessGroupSummary(item.id);
+
+                    // hide the summary loading indicator
+                    $('#summary-loading-container').hide();
+
+                    // show the dialog
+                    $('#cluster-remote-process-group-summary-dialog').modal('show');
+                }
+            }
+        });
+
         // wire up the dataview to the grid
         remoteProcessGroupsData.onRowCountChanged.subscribe(function (e, args) {
             remoteProcessGroupsGrid.updateRowCount();
@@ -1283,6 +1443,11 @@ nf.SummaryTable = (function () {
                 }
             }
         });
+        
+        // cluster remote process group refresh
+        nf.Common.addHoverEffect('#cluster-remote-process-group-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
+            loadClusterRemoteProcessGroupSummary($('#cluster-remote-process-group-id').text());
+        });
 
         // initialize the cluster remote process group column model
         var clusterRemoteProcessGroupsColumnModel = [
@@ -1749,7 +1914,6 @@ nf.SummaryTable = (function () {
 
                 // populate the processor details
                 $('#cluster-processor-name').text(clusterProcessorStatus.processorName).ellipsis();
-                ;
                 $('#cluster-processor-id').text(clusterProcessorStatus.processorId);
 
                 // update the stats last refreshed timestamp
@@ -1801,7 +1965,6 @@ nf.SummaryTable = (function () {
 
                 // populate the processor details
                 $('#cluster-connection-name').text(clusterConnectionStatus.connectionName).ellipsis();
-                ;
                 $('#cluster-connection-id').text(clusterConnectionStatus.connectionId);
 
                 // update the stats last refreshed timestamp
@@ -1959,18 +2122,16 @@ nf.SummaryTable = (function () {
             }
         }).fail(nf.Common.handleAjaxError);
     };
-    
+
     return {
         /**
          * URL for loading system diagnostics.
          */
         systemDiagnosticsUrl: null,
-        
         /**
          * URL for loading the summary.
          */
         url: null,
-        
         /**
          * Initializes the status table.
          * 
@@ -1994,7 +2155,6 @@ nf.SummaryTable = (function () {
                 });
             }).promise();
         },
-        
         /**
          * Update the size of the grid based on its container's current size.
          */
@@ -2024,7 +2184,6 @@ nf.SummaryTable = (function () {
                 remoteProcessGroupGrid.resizeCanvas();
             }
         },
-        
         /**
          * Load the processor status table.
          */
@@ -2043,7 +2202,7 @@ nf.SummaryTable = (function () {
                     // remove any tooltips from the processor table
                     var processorsGridElement = $('#processor-summary-table');
                     nf.Common.cleanUpTooltips(processorsGridElement, 'img.has-bulletins');
-                    
+
                     // get the processor grid/data
                     var processorsGrid = processorsGridElement.data('gridInstance');
                     var processorsData = processorsGrid.getData();
@@ -2129,308 +2288,6 @@ nf.SummaryTable = (function () {
                     $('#total-items').text('0');
                 }
             }).fail(nf.Common.handleAjaxError);
-        },
-        
-        // processor actions
-
-        /**
-         * Shows the details for the processor at the specified row.
-         * 
-         * @param {type} row        row index
-         */
-        showProcessorDetails: function (row) {
-            var grid = $('#processor-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.ProcessorDetails.showDetails(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Goes to the specified processor.
-         * 
-         * @param {type} row
-         */
-        goToProcessor: function (row) {
-            var grid = $('#processor-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                goTo(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the processor status history for a cluster.
-         * 
-         * @param {type} row
-         */
-        showClusterProcessorStatusHistory: function (row) {
-            var grid = $('#processor-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.StatusHistory.showClusterProcessorChart(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the processor status history.
-         * 
-         * @param {type} row
-         */
-        showProcessorStatusHistory: function (row) {
-            var grid = $('#processor-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.StatusHistory.showStandaloneProcessorChart(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the cluster processor details dialog for the specified processor.
-         * 
-         * @argument {string} row     The row
-         */
-        showClusterProcessorSummary: function (row) {
-            var grid = $('#processor-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // load the cluster processor summary
-                loadClusterProcessorSummary(item.id);
-
-                // hide the summary loading indicator
-                $('#summary-loading-container').hide();
-
-                // show the dialog
-                $('#cluster-processor-summary-dialog').modal('show');
-            }
-        },
-        
-        // connection actions
-
-        /**
-         * Shows the details for the connection at the specified row.
-         * 
-         * @param {type} row        row index
-         */
-        showConnectionDetails: function (row) {
-            var grid = $('#connection-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.ConnectionDetails.showDetails(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Goes to the specified connection.
-         * 
-         * @param {type} row
-         */
-        goToConnection: function (row) {
-            var grid = $('#connection-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                goTo(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the connection status history for a cluster.
-         * 
-         * @param {type} row
-         */
-        showClusterConnectionStatusHistory: function (row) {
-            var grid = $('#connection-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.StatusHistory.showClusterConnectionChart(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the connection status history.
-         * 
-         * @param {type} row
-         */
-        showConnectionStatusHistory: function (row) {
-            var grid = $('#connection-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.StatusHistory.showStandaloneConnectionChart(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the cluster connection details dialog for the specified connection.
-         * 
-         * @argument {string} row     The row
-         */
-        showClusterConnectionSummary: function (row) {
-            var grid = $('#connection-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // load the cluster processor summary
-                loadClusterConnectionSummary(item.id);
-
-                // hide the summary loading indicator
-                $('#summary-loading-container').hide();
-
-                // show the dialog
-                $('#cluster-connection-summary-dialog').modal('show');
-            }
-        },
-        
-        // input actions
-
-        /**
-         * Goes to the specified input port.
-         * 
-         * @param {type} row
-         */
-        goToInputPort: function (row) {
-            var grid = $('#input-port-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                goTo(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the cluster input port details dialog for the specified connection.
-         * 
-         * @argument {string} row     The row
-         */
-        showClusterInputPortSummary: function (row) {
-            var grid = $('#input-port-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // load the cluster processor summary
-                loadClusterInputPortSummary(item.id);
-
-                // hide the summary loading indicator
-                $('#summary-loading-container').hide();
-
-                // show the dialog
-                $('#cluster-input-port-summary-dialog').modal('show');
-            }
-        },
-        
-        // output actions
-
-        /**
-         * Goes to the specified output port.
-         * 
-         * @param {type} row
-         */
-        goToOutputPort: function (row) {
-            var grid = $('#output-port-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                goTo(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the cluster output port details dialog for the specified connection.
-         * 
-         * @argument {string} row     The row
-         */
-        showClusterOutputPortSummary: function (row) {
-            var grid = $('#output-port-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // load the cluster processor summary
-                loadClusterOutputPortSummary(item.id);
-
-                // hide the summary loading indicator
-                $('#summary-loading-container').hide();
-
-                // show the dialog
-                $('#cluster-output-port-summary-dialog').modal('show');
-            }
-        },
-        
-        // remote process group actions
-
-        /**
-         * Goes to the specified remote process group.
-         * 
-         * @param {type} row
-         */
-        goToRemoteProcessGroup: function (row) {
-            var grid = $('#remote-process-group-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                goTo(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the remote process group status history for a cluster.
-         * 
-         * @param {type} row
-         */
-        showClusterRemoteProcessGroupStatusHistory: function (row) {
-            var grid = $('#remote-process-group-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.StatusHistory.showClusterRemoteProcessGroupChart(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the remote process group status history.
-         * 
-         * @param {type} row
-         */
-        showRemoteProcessGroupStatusHistory: function (row) {
-            var grid = $('#remote-process-group-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-                nf.StatusHistory.showStandaloneRemoteProcessGroupChart(item.groupId, item.id);
-            }
-        },
-        
-        /**
-         * Shows the cluster remote process group details dialog for the specified connection.
-         * 
-         * @argument {string} row     The row
-         */
-        showClusterRemoteProcessGroupSummary: function (row) {
-            var grid = $('#remote-process-group-summary-table').data('gridInstance');
-            if (nf.Common.isDefinedAndNotNull(grid)) {
-                var data = grid.getData();
-                var item = data.getItem(row);
-
-                // load the cluster processor summary
-                loadClusterRemoteProcessGroupSummary(item.id);
-
-                // hide the summary loading indicator
-                $('#summary-loading-container').hide();
-
-                // show the dialog
-                $('#cluster-remote-process-group-summary-dialog').modal('show');
-            }
         }
     };
 }());
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b4d07e26/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary.js
index 7b90a0f..4f80241 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary.js
@@ -69,12 +69,6 @@ nf.Summary = (function () {
         nf.Common.addHoverEffect('#refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
             nf.SummaryTable.loadProcessorSummaryTable();
         });
-        nf.Common.addHoverEffect('#cluster-processor-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
-            nf.SummaryTable.loadClusterProcessorSummary($('#cluster-processor-id').text());
-        });
-        nf.Common.addHoverEffect('#cluster-connection-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
-            nf.SummaryTable.loadClusterConnectionSummary($('#cluster-connection-id').text());
-        });
 
         // return a deferred for page initialization
         return $.Deferred(function (deferred) {


[19/36] incubator-nifi git commit: NIFI-287: - Updating the way active thread count is displayed.

Posted by mc...@apache.org.
NIFI-287:
- Updating the way active thread count is displayed.

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

Branch: refs/heads/NIFI-250
Commit: d145e2098fcd7f0b1165e0ed37e93ab9482728b2
Parents: 9c61bdb
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Feb 3 09:43:01 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Feb 3 09:43:01 2015 -0500

----------------------------------------------------------------------
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js    | 17 ++++++++++++++---
 .../main/webapp/js/nf/canvas/nf-process-group.js   |  6 +++---
 .../webapp/js/nf/canvas/nf-remote-process-group.js |  2 +-
 3 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d145e209/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
index 40116ca..83a3117 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
@@ -349,7 +349,7 @@ nf.CanvasUtils = (function () {
                         .each(function () {
                             var bBox = this.getBBox();
                             d3.select(this).attr('x', function () {
-                                return d.dimensions.width - bBox.width - 2;
+                                return d.dimensions.width - bBox.width - 4;
                             });
                         });
 
@@ -357,7 +357,7 @@ nf.CanvasUtils = (function () {
                 selection.select('rect.active-thread-count-background')
                         .attr('width', function () {
                             var bBox = activeThreadCount.node().getBBox();
-                            return bBox.width + 4;
+                            return bBox.width + 8;
                         })
                         .attr('x', function () {
                             var bBox = activeThreadCount.node().getBBox();
@@ -367,7 +367,18 @@ nf.CanvasUtils = (function () {
                                 setOffset(bBox.width + 6);
                             }
 
-                            return d.dimensions.width - bBox.width - 4;
+                            return d.dimensions.width - bBox.width - 8;
+                        })
+                        .attr('stroke-dasharray', function() {
+                            var rect = d3.select(this);
+                            var width = parseFloat(rect.attr('width'));
+                            var height = parseFloat(rect.attr('height'));
+                            
+                            var dashArray = [];
+                            dashArray.push(0);
+                            dashArray.push(width + height);
+                            dashArray.push(width + height);
+                            return dashArray.join(' ');
                         })
                         .style('display', 'block');
             } else {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d145e209/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group.js
index 0bd522c..ae42547 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-process-group.js
@@ -565,9 +565,9 @@ nf.ProcessGroup = (function () {
                             .attr({
                                 'class': 'active-thread-count-background',
                                 'height': 13,
-                                'y': 2,
+                                'y': 0,
                                 'fill': '#fff',
-                                'fill-opacity': '0.65',
+                                'fill-opacity': '0.4',
                                 'stroke': '#aaa',
                                 'stroke-width': '1'
                             });
@@ -577,7 +577,7 @@ nf.ProcessGroup = (function () {
                             .attr({
                                 'class': 'active-thread-count',
                                 'height': 13,
-                                'y': 12,
+                                'y': 10,
                                 'fill': '#000'
                             });
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d145e209/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
index e55f5bb..6c4a302 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
@@ -512,7 +512,7 @@ nf.RemoteProcessGroup = (function () {
                                 'height': 13,
                                 'y': 0,
                                 'fill': '#fff',
-                                'fill-opacity': '0.65',
+                                'fill-opacity': '0.4',
                                 'stroke': '#aaa',
                                 'stroke-width': '1'
                             });


[25/36] incubator-nifi git commit: NIFI-268: Change Assert.fail to logger.warn if using deprecated annotation for processors. This will be cahnged to Assert.fail in 0.1.0 but should not happen for version 0.0.2

Posted by mc...@apache.org.
NIFI-268: Change Assert.fail to logger.warn if using deprecated annotation for processors. This will be cahnged to Assert.fail in 0.1.0 but should not happen for version 0.0.2


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

Branch: refs/heads/NIFI-250
Commit: 3b399d75044ba2bc52d6876e7fe295f85b9a1264
Parents: 10714ef
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 3 12:09:42 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 3 12:09:42 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/util/StandardProcessorTestRunner.java     | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3b399d75/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 63d1c9f..f3fd3e8 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -66,6 +66,8 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.ProvenanceReporter;
 import org.apache.nifi.reporting.InitializationException;
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class StandardProcessorTestRunner implements TestRunner {
 
@@ -80,6 +82,7 @@ public class StandardProcessorTestRunner implements TestRunner {
     private int numThreads = 1;
     private final AtomicInteger invocations = new AtomicInteger(0);
 
+    private static final Logger logger = LoggerFactory.getLogger(StandardProcessorTestRunner.class);
     private static final Set<Class<? extends Annotation>> deprecatedTypeAnnotations = new HashSet<>();
     private static final Set<Class<? extends Annotation>> deprecatedMethodAnnotations = new HashSet<>();
     
@@ -134,14 +137,14 @@ public class StandardProcessorTestRunner implements TestRunner {
     private static void detectDeprecatedAnnotations(final Processor processor) {
         for ( final Class<? extends Annotation> annotationClass : deprecatedTypeAnnotations ) {
             if ( processor.getClass().isAnnotationPresent(annotationClass) ) {
-                Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName());
+                logger.warn("Processor is using deprecated Annotation " + annotationClass.getCanonicalName());
             }
         }
         
         for ( final Class<? extends Annotation> annotationClass : deprecatedMethodAnnotations ) {
             for ( final Method method : processor.getClass().getMethods() ) {
                 if ( method.isAnnotationPresent(annotationClass) ) {
-                    Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName() + " for method " + method);
+                    logger.warn("Processor is using deprecated Annotation " + annotationClass.getCanonicalName() + " for method " + method);
                 }
             }
         }


[31/36] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


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

Branch: refs/heads/NIFI-250
Commit: 629ec371cdca232c7f40991c35dc628590c11c6f
Parents: bbb7f8d b0b14ed
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 4 08:02:41 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 4 08:02:41 2015 -0500

----------------------------------------------------------------------
 .gitignore                                    |   7 +
 nifi-site/src/includes/topbar.hbs             |   7 +-
 nifi-site/src/layouts/markdown.hbs            |   1 +
 nifi-site/src/pages/html/download.hbs         |  46 +++++
 nifi-site/src/pages/markdown/quickstart.md    |  14 +-
 nifi-site/src/pages/markdown/release-guide.md | 208 +++++++++++----------
 nifi-site/src/scss/_settings.scss             |  11 +-
 nifi-site/src/scss/app.scss                   |   8 +
 8 files changed, 197 insertions(+), 105 deletions(-)
----------------------------------------------------------------------



[12/36] incubator-nifi git commit: NIFI-288: - Removing the border radius on the provenance progress bars.

Posted by mc...@apache.org.
NIFI-288:
- Removing the border radius on the provenance progress bars.

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

Branch: refs/heads/NIFI-250
Commit: ed8f7716061055e4039b1586a4c7ff41f6c3e06a
Parents: 93b78a7
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 14:39:49 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 14:39:49 2015 -0500

----------------------------------------------------------------------
 .../nifi-web/nifi-web-ui/src/main/webapp/css/provenance.css     | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ed8f7716/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/provenance.css
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/provenance.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/provenance.css
index 7877f69..751a647 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/provenance.css
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/provenance.css
@@ -453,6 +453,11 @@ div.content-detail-value {
 
 #provenance-percent-complete, #lineage-percent-complete {
     width: 378px;
+    border-radius: 0;
+}
+
+#provenance-percent-complete .ui-progressbar-value, #lineage-percent-complete .ui-progressbar-value {
+    border-radius: 0;
 }
 
 div.progress-label {


[15/36] incubator-nifi git commit: NIFI-162: - Initial commit of the updated website. Contains copies of current pages and links to existing documents.

Posted by mc...@apache.org.
NIFI-162:
- Initial commit of the updated website. Contains copies of current pages and links to existing documents.

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

Branch: refs/heads/NIFI-250
Commit: 9c61bdb1025d78d49907321f29f91e8dad922fa0
Parents: 54e922c
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 2 23:21:21 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 2 23:21:21 2015 -0500

----------------------------------------------------------------------
 nifi-site/Gruntfile.js                          |  137 ++
 nifi-site/LICENSE                               |  202 +++
 nifi-site/README.md                             |  105 ++
 nifi-site/bower.json                            |   19 +
 nifi-site/config.rb                             |   24 +
 nifi-site/package.json                          |   24 +
 nifi-site/robots.txt                            |    4 +
 nifi-site/src/images/bgNifiLogo.png             |  Bin 0 -> 4232 bytes
 nifi-site/src/images/egg-logo.png               |  Bin 0 -> 8626 bytes
 nifi-site/src/images/feather-small.gif          |  Bin 0 -> 7129 bytes
 nifi-site/src/images/flow.png                   |  Bin 0 -> 503302 bytes
 .../src/images/niFi-logo-horizontal-scaled.png  |  Bin 0 -> 4462 bytes
 nifi-site/src/images/niFi-logo-horizontal.png   |  Bin 0 -> 13161 bytes
 nifi-site/src/images/nifi16.ico                 |  Bin 0 -> 1150 bytes
 nifi-site/src/images/nifiDrop.svg               |  217 +++
 nifi-site/src/includes/disclaimer.hbs           |   27 +
 nifi-site/src/includes/footer.hbs               |    6 +
 nifi-site/src/includes/header.hbs               |   22 +
 nifi-site/src/includes/topbar.hbs               |   65 +
 nifi-site/src/js/app.js                         |   15 +
 nifi-site/src/layouts/html.hbs                  |    3 +
 nifi-site/src/layouts/markdown.hbs              |   11 +
 nifi-site/src/pages/html/faq.hbs                |   33 +
 nifi-site/src/pages/html/index.hbs              |  210 +++
 nifi-site/src/pages/html/mailing_lists.hbs      |   70 +
 nifi-site/src/pages/html/people.hbs             |  136 ++
 nifi-site/src/pages/html/roadmap.hbs            |   30 +
 nifi-site/src/pages/markdown/quickstart.md      |   95 ++
 nifi-site/src/pages/markdown/release-guide.md   |  286 ++++
 nifi-site/src/scss/_settings.scss               | 1458 ++++++++++++++++++
 nifi-site/src/scss/app.scss                     |  177 +++
 31 files changed, 3376 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/Gruntfile.js
----------------------------------------------------------------------
diff --git a/nifi-site/Gruntfile.js b/nifi-site/Gruntfile.js
new file mode 100644
index 0000000..9139200
--- /dev/null
+++ b/nifi-site/Gruntfile.js
@@ -0,0 +1,137 @@
+module.exports = function (grunt) {
+    // Project configuration.
+    grunt.initConfig({
+        pkg: grunt.file.readJSON('package.json'),
+        
+        clean: {
+            options: {
+                force: true
+            },
+            js: ['dist/js/'],
+            css: ['dist/css/'],
+            assets: ['dist/assets/*']
+        },
+        
+        assemble: {
+            options: {
+                partials: 'src/includes/*.hbs',
+                layout: 'src/layouts/html.hbs',
+                flatten: true
+            },
+            html: {
+                files: {
+                    'dist/': ['src/pages/html/*.hbs']
+                }
+            },
+            markdown: {
+                options: {
+                    layout: 'src/layouts/markdown.hbs'
+                },
+                files: {
+                    'dist/': ['src/pages/markdown/*.md']
+                }
+            }
+        },
+        
+        compass: {
+            dist: {
+                options: {
+                    config: 'config.rb'
+                }
+            }
+        },
+        
+        concat: {
+            options: {
+                separator: ';'
+            },
+            foundation: {
+                src: [
+                    'bower_components/foundation/js/foundation/foundation.js',
+                    'bower_components/foundation/js/foundation/foundation.topbar.js'
+                ],
+                dest: 'dist/assets/js/foundation.js'
+            },
+            modernizr: {
+                src: [
+                    'bower_components/modernizr/modernizr.js'
+                ],
+                dest: 'dist/assets/js/modernizr.js'
+            },
+            nifi: {
+                src: [
+                    'src/js/app.js'
+                ],
+                dest: 'dist/js/app.js'
+            }
+        },
+        
+        copy: {
+            dist: {
+                files: [{
+                    expand: true,
+                    cwd:  'src/images/',
+                    src:  ['**/*.{png,jpg,gif,svg,ico}'],
+                    dest: 'dist/images/'
+                }, {
+                    expand: true,
+                    cwd:  'bower_components/jquery/dist',
+                    src:  ['jquery.min.js'],
+                    dest: 'dist/assets/js/'
+                }, {
+                    expand: true,
+                    cwd:  'bower_components/webfontloader',
+                    src:  ['webfontloader.js'],
+                    dest: 'dist/assets/js/'
+                }, {
+                    expand: true,
+                    cwd:  'bower_components/font-awesome/css',
+                    src:  ['font-awesome.min.css'],
+                    dest: 'dist/assets/stylesheets/'
+                }, {
+                    expand: true,
+                    cwd:  'bower_components/font-awesome',
+                    src:  ['fonts/*'],
+                    dest: 'dist/assets/'
+                }]
+            }
+        },
+        
+        watch: {
+            grunt: { 
+                files: ['Gruntfile.js'], 
+                tasks: ['dev'] 
+            },
+            css: {
+                files: 'src/scss/*.scss',
+                tasks: ['css']
+            },
+            script: { 
+                files: 'src/js/*.js',
+                tasks: ['js']
+            },
+            images: {
+                files: 'src/images/*.{png,jpg,gif,svg,ico}',
+                tasks: ['img']
+            },
+            assemble: {
+                files: ['src/{includes,layouts}/*.hbs', 'src/pages/{html,markdown}/*.{hbs,md}'],
+                tasks: ['assemble']
+            }
+        }
+    });
+    
+    grunt.loadNpmTasks('grunt-newer');
+    grunt.loadNpmTasks('grunt-contrib-clean');
+    grunt.loadNpmTasks('grunt-contrib-copy');
+    grunt.loadNpmTasks('grunt-contrib-concat');
+    grunt.loadNpmTasks('assemble');
+    grunt.loadNpmTasks('grunt-contrib-compass');
+    grunt.loadNpmTasks('grunt-contrib-watch');
+    
+    grunt.registerTask('img', ['newer:copy']);
+    grunt.registerTask('css', ['clean:css', 'compass']);
+    grunt.registerTask('js',  ['clean:js', 'concat']);
+    grunt.registerTask('dev',  ['clean', 'assemble', 'css', 'js', 'img', 'copy', 'watch']);
+    grunt.registerTask('default', ['clean', 'assemble', 'css', 'js', 'img', 'copy']);
+};

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/LICENSE
----------------------------------------------------------------------
diff --git a/nifi-site/LICENSE b/nifi-site/LICENSE
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/nifi-site/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/README.md
----------------------------------------------------------------------
diff --git a/nifi-site/README.md b/nifi-site/README.md
new file mode 100644
index 0000000..45276e1
--- /dev/null
+++ b/nifi-site/README.md
@@ -0,0 +1,105 @@
+# About
+[Apache NiFi project] (http://nifi.incubator.apache.org).
+
+## Getting Started
+
+The site is built using [grunt][] task runner, [bower][] package manager for
+the web, and [npm][] package manager for node. npm is used to manage the
+node modules required for building this site. bower is used to manage the 
+front end packages that the site will depend on.
+
+Both grunt and bower can be installed via npm once it is installed. 
+
+```bash
+sudo npm install -g grunt-cli
+sudo npm install -g bower
+```
+
+To download the front end packages required for this site run the following
+command from the nifi-site directory.
+
+```bash
+bower install
+```
+
+To install the node modules required to build this site run the following
+command from the nifi-site directory.
+
+```bash
+npm install
+```
+
+The site is built using [foundation][] a responsive front end framework. 
+Consequently, the site is using [sass][] and [compass][] for CSS pre-processing.
+This will also require ruby to be installed along with sass and compass. Both
+sass and compass can be installed via ruby once it is installed.
+
+```bash
+gem install compass
+```
+
+[grunt]: http://gruntjs.com/
+[bower]: http://bower.io/
+[npm]: http://www.npmjs.com/
+[foundation]: http://foundation.zurb.com/
+[sass]: http://sass-lang.com/
+[compass]: http://compass-style.org/
+
+## Grunt Tasks
+
+To build the site run the default grunt task. This will assemble the site and 
+place the resulting site in the dist folder.
+
+```bash
+grunt
+```
+
+If developing new content/features for the site, it may be helpful to run
+the dev task which will build the site and continue to watch the source
+files for changes. Any changes will cause the site to be rebuilt.
+
+```bash
+grunt dev
+```
+
+## Application Style Properties
+
+### Font
+- Oswald: http://www.google.com/fonts/specimen/Oswald
+
+### Colors
+- 'ni':  #7A96A4
+- 'fi':  #0F3541
+- Toolbox:  #7299AC
+- Toolbar:  #A0BAC7
+- Flow Status: #D9E4E8
+- Utilities: #81A0B4
+
+## Technologies & Resources
+- HTML5
+- CSS3
+- Foundation
+- jQuery
+- Modernizr
+- Google Fonts API
+- Web Font Loader
+- FontAwesome
+
+## License
+
+Except as otherwise noted this software is licensed under the
+[Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html)
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/bower.json
----------------------------------------------------------------------
diff --git a/nifi-site/bower.json b/nifi-site/bower.json
new file mode 100644
index 0000000..6ced6f7
--- /dev/null
+++ b/nifi-site/bower.json
@@ -0,0 +1,19 @@
+{
+  "name": "apache-nifi-site",
+  "version": "0.0.2-incubating",
+  "description": "The artifacts for the Apache NiFi site.",
+  "private": "true",
+  "repository": {
+    "type": "git",
+    "url": "http://git-wip-us.apache.org/repos/asf/incubator-nifi.git"
+  },
+  "dependencies": {
+    "foundation": "~5.5.1",
+    "font-awesome": "~4.3.0",
+    "webfontloader": "~1.5.14"
+  },
+  "licenses" : [{
+    "type": "Apache",
+    "url": "http://www.apache.org/licenses/LICENSE-2.0.html"
+  }]
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/config.rb
----------------------------------------------------------------------
diff --git a/nifi-site/config.rb b/nifi-site/config.rb
new file mode 100644
index 0000000..296860f
--- /dev/null
+++ b/nifi-site/config.rb
@@ -0,0 +1,24 @@
+# Require any additional compass plugins here.
+add_import_path "bower_components/foundation/scss"
+
+# Set this to the root of your project when deployed:
+http_path = "/"
+css_dir = "dist/stylesheets"
+sass_dir = "src/scss"
+images_dir = "dist/images"
+javascripts_dir = "dist/js"
+
+# You can select your preferred output style here (can be overridden via the command line):
+# output_style = :expanded or :nested or :compact or :compressed
+
+# To enable relative paths to assets via compass helper functions. Uncomment:
+# relative_assets = true
+
+# To disable debugging comments that display the original location of your selectors. Uncomment:
+# line_comments = false
+
+# If you prefer the indented syntax, you might want to regenerate this
+# project again passing --syntax sass, or you can uncomment this:
+# preferred_syntax = :sass
+# and then run:
+# sass-convert -R --from scss --to sass sass scss && rm -rf sass && mv scss sass

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/package.json
----------------------------------------------------------------------
diff --git a/nifi-site/package.json b/nifi-site/package.json
new file mode 100644
index 0000000..170d4ad
--- /dev/null
+++ b/nifi-site/package.json
@@ -0,0 +1,24 @@
+{
+  "name": "apache-nifi-site",
+  "version": "0.0.2-incubating",
+  "description": "The artifacts for the Apache NiFi site.",
+  "private": "true",
+  "repository": {
+    "type": "git",
+    "url": "http://git-wip-us.apache.org/repos/asf/incubator-nifi.git"
+  },
+  "devDependencies": {
+    "assemble": "^0.4.42",
+    "grunt": "^0.4.5",
+    "grunt-contrib-clean": "^0.6.0",
+    "grunt-contrib-compass": "^1.0.1",
+    "grunt-contrib-concat": "^0.5.0",
+    "grunt-contrib-copy": "^0.7.0",
+    "grunt-contrib-watch": "^0.6.1",
+    "grunt-newer": "^1.1.0"
+  },
+  "licenses" : [{
+    "type": "Apache",
+    "url": "http://www.apache.org/licenses/LICENSE-2.0.html"
+  }]
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/robots.txt
----------------------------------------------------------------------
diff --git a/nifi-site/robots.txt b/nifi-site/robots.txt
new file mode 100644
index 0000000..214e411
--- /dev/null
+++ b/nifi-site/robots.txt
@@ -0,0 +1,4 @@
+# www.robotstxt.org/
+# www.google.com/support/webmasters/bin/answer.py?hl=en&answer=156449
+
+User-agent: *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/bgNifiLogo.png
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/bgNifiLogo.png b/nifi-site/src/images/bgNifiLogo.png
new file mode 100644
index 0000000..d92c484
Binary files /dev/null and b/nifi-site/src/images/bgNifiLogo.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/egg-logo.png
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/egg-logo.png b/nifi-site/src/images/egg-logo.png
new file mode 100644
index 0000000..c04e70d
Binary files /dev/null and b/nifi-site/src/images/egg-logo.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/feather-small.gif
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/feather-small.gif b/nifi-site/src/images/feather-small.gif
new file mode 100644
index 0000000..fb4445f
Binary files /dev/null and b/nifi-site/src/images/feather-small.gif differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/flow.png
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/flow.png b/nifi-site/src/images/flow.png
new file mode 100644
index 0000000..2b48140
Binary files /dev/null and b/nifi-site/src/images/flow.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/niFi-logo-horizontal-scaled.png
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/niFi-logo-horizontal-scaled.png b/nifi-site/src/images/niFi-logo-horizontal-scaled.png
new file mode 100644
index 0000000..ebe723e
Binary files /dev/null and b/nifi-site/src/images/niFi-logo-horizontal-scaled.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/niFi-logo-horizontal.png
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/niFi-logo-horizontal.png b/nifi-site/src/images/niFi-logo-horizontal.png
new file mode 100644
index 0000000..d43a263
Binary files /dev/null and b/nifi-site/src/images/niFi-logo-horizontal.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/nifi16.ico
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/nifi16.ico b/nifi-site/src/images/nifi16.ico
new file mode 100644
index 0000000..2ac3670
Binary files /dev/null and b/nifi-site/src/images/nifi16.ico differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/images/nifiDrop.svg
----------------------------------------------------------------------
diff --git a/nifi-site/src/images/nifiDrop.svg b/nifi-site/src/images/nifiDrop.svg
new file mode 100644
index 0000000..7026f84
--- /dev/null
+++ b/nifi-site/src/images/nifiDrop.svg
@@ -0,0 +1,217 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!-- Generator: Adobe Illustrator 16.0.3, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<svg version="1.1" id="Layer_1" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" x="0px" y="0px"
+	 width="44px" height="59.76px" viewBox="0 0 44 59.76" enable-background="new 0 0 44 59.76" xml:space="preserve">
+<g>
+	<linearGradient id="SVGID_1_" gradientUnits="userSpaceOnUse" x1="13.8408" y1="19.4224" x2="26.1859" y2="50.6966">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<path fill="url(#SVGID_1_)" d="M27.186,36.189h6.752h4.927c0-8.761-6.873-17.16-11.563-23.287C22.786,7.01,20.428,2,20.428,2
+		s-2.349,5.01-6.866,10.903C8.865,19.029,2,27.429,2,36.189c0,10.178,8.252,18.435,18.428,18.435v-4.928v-6.749h6.758V36.189z"/>
+	<linearGradient id="SVGID_2_" gradientUnits="userSpaceOnUse" x1="13.7744" y1="19.4326" x2="26.1248" y2="50.7201">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<rect x="21.394" y="43.904" fill="url(#SVGID_2_)" width="6.751" height="6.758"/>
+	<linearGradient id="SVGID_3_" gradientUnits="userSpaceOnUse" x1="11.4067" y1="20.3711" x2="23.754" y2="51.6508">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<path fill="url(#SVGID_3_)" d="M28.145,51.758h-6.751v4.928c2.383,0,4.659-0.455,6.751-1.277V51.758z"/>
+	<linearGradient id="SVGID_4_" gradientUnits="userSpaceOnUse" x1="21.9297" y1="16.2295" x2="34.2729" y2="47.499">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<rect x="28.145" y="37.154" fill="url(#SVGID_4_)" width="6.756" height="6.75"/>
+	<linearGradient id="SVGID_5_" gradientUnits="userSpaceOnUse" x1="20.3535" y1="16.8696" x2="32.6938" y2="48.1316">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<path fill="url(#SVGID_5_)" d="M29.529,45.291v6.751h5.783c0.332-0.308,0.66-0.636,0.969-0.97v-5.781H29.529z"/>
+	<linearGradient id="SVGID_6_" gradientUnits="userSpaceOnUse" x1="18.7891" y1="17.4702" x2="31.1328" y2="48.741">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<path fill="url(#SVGID_6_)" d="M31.077,57.76c2.163-0.848,4.116-2.098,5.791-3.649h-5.791V57.76z"/>
+	<linearGradient id="SVGID_7_" gradientUnits="userSpaceOnUse" x1="27.9355" y1="13.8599" x2="40.2783" y2="45.128">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<path fill="url(#SVGID_7_)" d="M35.999,43.904h3.647c0.825-2.09,1.28-4.366,1.28-6.75h-4.928V43.904z"/>
+	<linearGradient id="SVGID_8_" gradientUnits="userSpaceOnUse" x1="26.2617" y1="14.4844" x2="38.6173" y2="45.7851">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.2319" style="stop-color:#7591A0"/>
+		<stop  offset="0.4181" style="stop-color:#6B8897"/>
+		<stop  offset="0.5885" style="stop-color:#5A7A88"/>
+		<stop  offset="0.7494" style="stop-color:#426573"/>
+		<stop  offset="0.9023" style="stop-color:#244A58"/>
+		<stop  offset="1" style="stop-color:#0C3543"/>
+	</linearGradient>
+	<path fill="url(#SVGID_8_)" d="M38.352,52.629C39.9,50.955,41.15,49.002,42,46.836h-3.648V52.629z"/>
+	<linearGradient id="SVGID_9_" gradientUnits="userSpaceOnUse" x1="22.7119" y1="53.0615" x2="16.3813" y2="5.1998">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_9_)" d="M26.775,35.721h6.348h4.629c0-8.242-6.448-16.133-10.862-21.895
+		c-4.247-5.544-6.462-10.249-6.462-10.249s-2.207,4.705-6.454,10.249C9.56,19.588,3.104,27.479,3.104,35.721
+		c0,9.569,7.345,17.324,16.914,17.324v-4.629v-5.938h6.758V35.721z"/>
+	<linearGradient id="SVGID_10_" gradientUnits="userSpaceOnUse" x1="25.4878" y1="52.6904" x2="19.1612" y2="4.8591">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<rect x="21.834" y="44.348" fill="url(#SVGID_10_)" width="5.876" height="5.871"/>
+	<linearGradient id="SVGID_11_" gradientUnits="userSpaceOnUse" x1="24.6118" y1="52.8105" x2="18.2843" y2="4.9722">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_11_)" d="M27.71,52.084h-5.876v3.871c2.07,0,4.057-0.391,5.876-1.113V52.084z"/>
+	<linearGradient id="SVGID_12_" gradientUnits="userSpaceOnUse" x1="33.0015" y1="51.6997" x2="26.6722" y2="3.8483">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<rect x="28.588" y="37.596" fill="url(#SVGID_12_)" width="5.873" height="5.874"/>
+	<linearGradient id="SVGID_13_" gradientUnits="userSpaceOnUse" x1="33.2969" y1="51.6577" x2="26.9677" y2="3.8071">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_13_)" d="M29.964,45.726V51.6h5.034c0.293-0.269,0.571-0.547,0.84-0.84v-5.034H29.964z"/>
+	<linearGradient id="SVGID_14_" gradientUnits="userSpaceOnUse" x1="32.7988" y1="51.7354" x2="26.4825" y2="3.9818">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_14_)" d="M31.457,57.109c1.878-0.739,2.349-0.998,3.795-2.348h-3.795V57.109z"/>
+	<linearGradient id="SVGID_15_" gradientUnits="userSpaceOnUse" x1="39.8223" y1="50.7959" x2="33.4946" y2="2.9561">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_15_)" d="M36.326,43.469h3.167c0.72-1.82,1.111-3.803,1.111-5.874h-4.278V43.469z"/>
+	<linearGradient id="SVGID_16_" gradientUnits="userSpaceOnUse" x1="40.3608" y1="50.7236" x2="34.035" y2="2.8982">
+		<stop  offset="0" style="stop-color:#7894A3"/>
+		<stop  offset="0.0281" style="stop-color:#849DAA"/>
+		<stop  offset="0.1087" style="stop-color:#A0B1BB"/>
+		<stop  offset="0.2" style="stop-color:#B7C2C8"/>
+		<stop  offset="0.3046" style="stop-color:#C8CFD3"/>
+		<stop  offset="0.4308" style="stop-color:#D5D8DA"/>
+		<stop  offset="0.6005" style="stop-color:#DCDDDF"/>
+		<stop  offset="1" style="stop-color:#DEDFE0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_16_)" d="M39.002,51.426c1.172-1.693,1.609-2.33,2.347-4.21h-2.347V51.426z"/>
+	<linearGradient id="SVGID_17_" gradientUnits="userSpaceOnUse" x1="10.2578" y1="24.1812" x2="21.0127" y2="28.7522">
+		<stop  offset="0" style="stop-color:#FFFFFF;stop-opacity:0.85"/>
+		<stop  offset="1" style="stop-color:#FFFFFF;stop-opacity:0"/>
+	</linearGradient>
+	<path fill="url(#SVGID_17_)" d="M20.018,42.479h6.758v-6.758h4.294c0.022-0.226,0.036-0.449,0.042-0.667
+		c0.234-6.033-2.54-12.666-5.106-16.809c-3.652-5.901-5.627-12.72-5.627-12.72s-2.272,4.228-5.426,8.257
+		C9.883,20.234,4.769,28.494,4.764,34.529c-0.007,8.965,5.29,14.277,12.298,14.55c0.998,0.037,1.988-0.084,2.956-0.332v-0.332
+		V42.479z"/>
+	<path fill="none" d="M26.775,35.721h6.348h4.629c0-8.242-6.448-16.133-10.862-21.895c-4.247-5.544-6.462-10.249-6.462-10.249
+		s-2.207,4.705-6.454,10.249C9.56,19.588,3.104,27.479,3.104,35.721c0,9.569,7.345,17.324,16.914,17.324v-4.629v-5.938h6.758V35.721
+		z"/>
+	<g>
+		<defs>
+			<path id="SVGID_18_" d="M20.018,42.518h6.758v-6.76h4.294c0.022-0.227,0.036-0.447,0.042-0.67
+				c0.234-6.031-2.54-12.665-5.106-16.807c-3.652-5.899-5.627-12.718-5.627-12.718s-2.272,4.226-5.426,8.254
+				C9.883,20.271,4.769,28.533,4.764,34.568c-0.007,8.965,5.29,14.277,12.298,14.545c0.998,0.037,1.988-0.084,2.956-0.327v-0.336
+				V42.518z"/>
+		</defs>
+		<clipPath id="SVGID_19_">
+			<use xlink:href="#SVGID_18_"  overflow="visible"/>
+		</clipPath>
+		<g opacity="0.75" clip-path="url(#SVGID_19_)">
+			<linearGradient id="SVGID_20_" gradientUnits="userSpaceOnUse" x1="7.6851" y1="40.1489" x2="27.6449" y2="35.9352">
+				<stop  offset="0" style="stop-color:#FFFFFF"/>
+				<stop  offset="0.9104" style="stop-color:#FFFFFF;stop-opacity:0.0896"/>
+				<stop  offset="1" style="stop-color:#FFFFFF;stop-opacity:0"/>
+			</linearGradient>
+			<path fill="url(#SVGID_20_)" d="M11.17,37.201C1.639,41.6-4.631,48.801-8.654,59.998c0.906-22.021,11.348-32.383,21.426-37.324
+				C10.096,30.873,11.17,37.201,11.17,37.201z"/>
+			<linearGradient id="SVGID_21_" gradientUnits="userSpaceOnUse" x1="5.8086" y1="31.2808" x2="25.7705" y2="27.0666">
+				<stop  offset="0" style="stop-color:#FFFFFF"/>
+				<stop  offset="0.9104" style="stop-color:#FFFFFF;stop-opacity:0.0896"/>
+				<stop  offset="1" style="stop-color:#FFFFFF;stop-opacity:0"/>
+			</linearGradient>
+			<path fill="url(#SVGID_21_)" d="M14.901,21.963c-2.67,8.193-1.6,14.521-1.6,14.521c5.67-2.204,14.253-3.093,19.333-3.303
+				c-1.239-7.207-4.009-13.729-4.009-13.729S21.954,19.939,14.901,21.963z"/>
+		</g>
+	</g>
+</g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/includes/disclaimer.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/includes/disclaimer.hbs b/nifi-site/src/includes/disclaimer.hbs
new file mode 100644
index 0000000..ce21060
--- /dev/null
+++ b/nifi-site/src/includes/disclaimer.hbs
@@ -0,0 +1,27 @@
+<div class="row">
+    <div class="large-12 columns footer">
+        <div class="disclaimer">
+            <p>
+                Disclaimer: Apache NiFi is an effort undergoing incubation at the Apache Software Foundation (ASF),
+                sponsored by the Apache Incubator PMC. Incubation is required of all newly accepted projects until 
+                a further review indicates that the infrastructure, communications, and decision making process 
+                have stabilized in a manner consistent with other successful ASF projects. While incubation status 
+                is not necessarily a reflection of the completeness or stability of the code, it does indicate that 
+                the project has yet to be fully endorsed by the ASF.
+            </p>
+        </div>
+        <a href="http://www.apache.org">
+            <img id="asf-logo" alt="Apache Software Foundation" src="images/feather-small.gif">
+        </a>
+        <a href="http://incubator.apache.org/">
+            <img id="incubator-logo" alt="Apache Incubator" src="images/egg-logo.png">
+        </a>
+
+        <div id="copyright">
+            <p>Copyright &#169; 2014 The Apache Software Foundation, Licensed under the <a
+                    href="http://www.apache.org/licenses/LICENSE-2.0">Apache License, Version 2.0</a>.<br/>Apache, the
+                Apache feather logo, and the Apache Incubator project logo are trademarks of The Apache Software
+                Foundation.</p>
+        </div>
+    </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/includes/footer.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/includes/footer.hbs b/nifi-site/src/includes/footer.hbs
new file mode 100644
index 0000000..cc9405a
--- /dev/null
+++ b/nifi-site/src/includes/footer.hbs
@@ -0,0 +1,6 @@
+        {{> disclaimer }}
+        <script src="assets/js/jquery.min.js"></script>
+        <script src="assets/js/foundation.js"></script>
+        <script src="js/app.js"></script>
+    </body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/includes/header.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/includes/header.hbs b/nifi-site/src/includes/header.hbs
new file mode 100644
index 0000000..5940e6b
--- /dev/null
+++ b/nifi-site/src/includes/header.hbs
@@ -0,0 +1,22 @@
+<!doctype html>
+<html class="no-js" lang="en">
+    <head>
+        <title>{{title}}</title>
+        <meta charset="utf-8" />
+        <meta name="viewport" content="width=device-width, initial-scale=1.0" />
+        <link rel="shortcut icon" href="images/nifi16.ico"/>
+        <link rel="stylesheet" href="stylesheets/app.css" />
+        <link rel="stylesheet" href="assets/stylesheets/font-awesome.min.css">
+        <script src="assets/js/modernizr.js"></script>
+        <script src="assets/js/webfontloader.js"></script>
+        <script>
+            (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
+            (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
+            m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
+            })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
+            ga('create', 'UA-57264262-1', 'auto');
+            ga('send', 'pageview');
+      </script>
+    </head>
+    <body>
+        {{> topbar}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/includes/topbar.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/includes/topbar.hbs b/nifi-site/src/includes/topbar.hbs
new file mode 100644
index 0000000..bbf7821
--- /dev/null
+++ b/nifi-site/src/includes/topbar.hbs
@@ -0,0 +1,65 @@
+<div class="sticky contain-to-grid">
+    <nav class="top-bar" data-topbar role="navigation">
+        <ul class="title-area">
+            <li class="name">
+                <h1>
+                    <a href="index.html">
+                        <img id="logo-top-bar" src="images/nifiDrop.svg" alt="Apache NiFi"/>
+                    </a>
+                </h1>
+            </li>
+            <!-- Remove the class "menu-icon" to get rid of menu icon. Take out "Menu" to just have icon alone -->
+            <li class="toggle-topbar menu-icon"><a href="#"><span></span></a></li>
+        </ul>
+
+        <section class="top-bar-section">
+            <!-- Right Nav Section -->
+            <ul class="right">
+                <li class="has-dropdown">
+                    <a href="#">Project</a>
+                    <ul class="dropdown">
+                        <li><a href="index.html">Home</a></li>
+                        <li><a href="https://blogs.apache.org/nifi/"><i class="fa fa-external-link external-link"></i>Apache NiFi Blog</a></li>
+                        <li><a href="#">Downloads</a></li>
+                        <li><a href="roadmap.html">Roadmap</a></li>
+                        <li><a href="http://www.apache.org/licenses/LICENSE-2.0"><i class="fa fa-external-link external-link"></i>License</a></li>
+                    </ul>
+                </li>
+                <li class="has-dropdown">
+                    <a href="#">Documentation</a>
+                    <ul class="dropdown">
+                        <li><a href="faq.html">FAQ</a></li>
+                        <li><a href="https://nifi.incubator.apache.org/docs/nifi-docs/overview.html"><i class="fa fa-external-link external-link"></i>NiFi Overview</a></li>
+                        <li><a href="https://nifi.incubator.apache.org/docs/nifi-docs/user-guide.html"><i class="fa fa-external-link external-link"></i>User Guide</a></li>
+                        <li><a href="https://nifi.incubator.apache.org/docs/nifi-docs/developer-guide.html"><i class="fa fa-external-link external-link"></i>Developer Guide</a></li>
+                    </ul>
+                </li>
+                <li class="has-dropdown">
+                    <a href="#">Community</a>
+                    <ul class="dropdown">
+                        <li><a href="mailing_lists.html">Mailing Lists</a></li>
+                        <li><a href="people.html">People</a></li>
+                    </ul>
+                </li>
+                <li class="has-dropdown">
+                    <a href="#">Development</a>
+                    <ul class="dropdown">
+                        <li><a href="quickstart.html">Quickstart</a></li>
+                        <li><a href="release-guide.html">Release Guide</a></li>
+                        <li><a href="https://git-wip-us.apache.org/repos/asf/incubator-nifi.git"><i class="fa fa-external-link external-link"></i>Source</a></li>
+                        <li><a href="https://issues.apache.org/jira/browse/NIFI"><i class="fa fa-external-link external-link"></i>Issues</a></li>
+                    </ul>
+                </li>
+                <li class="has-dropdown">
+                    <a href="#">ASF Links</a>
+                    <ul class="dropdown">
+                        <li><a href="http://www.apache.org"><i class="fa fa-external-link external-link"></i>Apache Software Foundation</a></li>
+                        <li><a href="http://www.apache.org/foundation/sponsorship.html"><i class="fa fa-external-link external-link"></i>Sponsorship</a></li>
+                        <li><a href="http://www.apache.org/security/"><i class="fa fa-external-link external-link"></i>Security</a></li>
+                        <li><a href="http://www.apache.org/foundation/thanks.html"><i class="fa fa-external-link external-link"></i>Thanks</a></li>
+                    </ul>
+                </li>
+            </ul>
+        </section>
+    </nav>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/js/app.js
----------------------------------------------------------------------
diff --git a/nifi-site/src/js/app.js b/nifi-site/src/js/app.js
new file mode 100644
index 0000000..9518bc0
--- /dev/null
+++ b/nifi-site/src/js/app.js
@@ -0,0 +1,15 @@
+// Foundation JavaScript
+// Documentation can be found at: http://foundation.zurb.com/docs
+$(document).foundation('topbar', {
+    mobile_show_parent_link: false,
+    is_hover: false
+});
+
+// load fonts
+$(document).ready(function() {
+    WebFont.load({
+        google: {
+            families: ['Oswald:400,700,300']
+        }
+    });
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/layouts/html.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/layouts/html.hbs b/nifi-site/src/layouts/html.hbs
new file mode 100644
index 0000000..6361bbb
--- /dev/null
+++ b/nifi-site/src/layouts/html.hbs
@@ -0,0 +1,3 @@
+{{> header }}
+{{> body }}
+{{> footer }}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/layouts/markdown.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/layouts/markdown.hbs b/nifi-site/src/layouts/markdown.hbs
new file mode 100644
index 0000000..fdfda90
--- /dev/null
+++ b/nifi-site/src/layouts/markdown.hbs
@@ -0,0 +1,11 @@
+{{> header }}
+
+<div class="row">
+    <div class="large-12 columns">
+        {{#markdown}}
+            {{> body }}
+        {{/markdown}}
+    </div>
+</div>
+
+{{> footer}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/html/faq.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/html/faq.hbs b/nifi-site/src/pages/html/faq.hbs
new file mode 100644
index 0000000..d4fd327
--- /dev/null
+++ b/nifi-site/src/pages/html/faq.hbs
@@ -0,0 +1,33 @@
+---
+title: Apache NiFi FAQs
+---
+
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h1 class="nifi-txt">
+            <span>
+                Apache <span class="ni">ni</span><span class="fi">fi</span> FAQs
+            </span>
+        </h1>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <ul>
+            <li>
+                <p class="description"><b>How do you pronounce NiFi?</b></p>
+                <p class="description">"nye fye" (nī fī) is the preferred pronunciation.</p>
+            </li>
+            <li>
+                <p class="description"><b>Where can I find documentation on how to understand and configure NiFi?</b></p>
+                <ul>
+                    <li>Our draft/preview of the User Guide is now available under the docs dropdown.</li>
+                    <li>We're working on completing that and having it fully integrated to the application.</li>
+                    <li>The developer guide is also in the works.</li>
+                    <li>For more information on presently outstanding documentation work <a href="https://issues.apache.org/jira/browse/NIFI-162?jql=project%20%3D%20NIFI%20AND%20resolution%20%3D%20Unresolved%20AND%20component%20%3D%20%22Documentation%20%26%20Website%22%20ORDER%20BY%20priority%20DESC">please click here</a>.</li>
+                </ul>
+            </li>
+        </ul>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/html/index.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/html/index.hbs b/nifi-site/src/pages/html/index.hbs
new file mode 100644
index 0000000..d01e8bc
--- /dev/null
+++ b/nifi-site/src/pages/html/index.hbs
@@ -0,0 +1,210 @@
+---
+title: Apache NiFi
+---
+
+<div class="large-space"></div>
+<div class="row">
+    <div class="medium-6 large-5 columns">
+        <div class="row">
+            <div class="large-12 columns">
+                <h1 id="nifi-landing" class="right-text nifi-txt">
+                    <span>
+                        Apache <span class="ni">ni</span><span class="fi">fi</span>
+                    </span>
+                </h1>
+            </div>
+        </div>
+        <div class="row">
+            <div class="large-12 columns">
+                <p class="right-text description">
+                    is an easy to use, powerful, and reliable system to process and distribute data.
+                </p>
+            </div>
+        </div>
+    </div>
+    <div class="medium-6 large-7 columns">
+        <img id="flow" src="images/flow.png" alt="NiFi flow"/>
+    </div>
+    <div class="clear"></div>
+</div>
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns features">
+        <h2>Features</h2>
+    </div>
+</div>
+<div class="medium-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <p class="description">
+            <span class="nifi-txt">Apache <span class="ni">ni</span><span class="fi">fi</span></span> supports
+            powerful and scalable directed graphs of data routing, transformation, and system mediation logic. Some
+            of the high-level capabilities and objectives of Apache NiFi include:
+        </p>
+        <ul>
+            <li>Web-based user interface for seamless experience between design, control, feedback, and monitoring
+                of data flows
+            </li>
+            <li>Highly configurable along several dimensions of quality of service such as loss tolerant versus
+                guaranteed delivery, low latency versus high throughput, and priority based queuing
+            </li>
+            <li>Fine-grained data provenance for all data received, forked, joined, cloned, modified, sent, and
+                ultimately dropped as data reaches its configured end-state
+            </li>
+            <li>Component-based extension model along well defined interfaces enabling rapid development and
+                effective testing
+            </li>
+            <li>Security: Supports various forms of SSL, SSH, HTTPS for system to system flows and content
+                encryption/decryption. And SSL, PKI, pluggable authorization for users.
+            </li>
+        </ul>
+    </div>
+</div>
+
+<!--<div class="row">
+    <div class="large-12 columns">
+        <h1>Welcome to Foundation</h1>
+    </div>
+</div>
+
+<div class="row">
+    <div class="large-12 columns">
+        <div class="panel">
+            <h3>We&rsquo;re stoked you want to try Foundation! </h3>
+            <p>To get going, this file (index.html) includes some basic styles you can modify, play around with, or totally destroy to get going.</p>
+            <p>Once you've exhausted the fun in this document, you should check out:</p>
+            <div class="row">
+                <div class="large-4 medium-4 columns">
+                    <p><a href="http://foundation.zurb.com/docs">Foundation Documentation</a><br />Everything you need to know about using the framework.</p>
+                </div>
+                <div class="large-4 medium-4 columns">
+                    <p><a href="http://github.com/zurb/foundation">Foundation on Github</a><br />Latest code, issue reports, feature requests and more.</p>
+                </div>
+                <div class="large-4 medium-4 columns">
+                    <p><a href="http://twitter.com/foundationzurb">@foundationzurb</a><br />Ping us on Twitter if you have questions. If you build something with this we'd love to see it (and send you a totally boss sticker).</p>
+                </div>        
+            </div>
+        </div>
+    </div>
+</div>
+
+<div class="row">
+    <div class="large-8 medium-8 columns">
+        <h5>Here&rsquo;s your basic grid:</h5>
+        Grid Example 
+
+        <div class="row">
+            <div class="large-12 columns">
+                <div class="callout panel">
+                    <p><strong>This is a twelve column section in a row.</strong> Each of these includes a div.panel element so you can see where the columns are - it's not required at all for the grid.</p>
+                </div>
+            </div>
+        </div>
+        <div class="row">
+            <div class="large-6 medium-6 columns">
+                <div class="callout panel">
+                    <p>Six columns</p>
+                </div>
+            </div>
+            <div class="large-6 medium-6 columns">
+                <div class="callout panel">
+                    <p>Six columns</p>
+                </div>
+            </div>
+        </div>
+        <div class="row">
+            <div class="large-4 medium-4 small-4 columns">
+                <div class="callout panel">
+                    <p>Four columns</p>
+                </div>
+            </div>
+            <div class="large-4 medium-4 small-4 columns">
+                <div class="callout panel">
+                    <p>Four columns</p>
+                </div>
+            </div>
+            <div class="large-4 medium-4 small-4 columns">
+                <div class="callout panel">
+                    <p>Four columns</p>
+                </div>
+            </div>
+        </div>
+
+        <hr />
+
+        <h5>We bet you&rsquo;ll need a form somewhere:</h5>
+        <form>
+            <div class="row">
+                <div class="large-12 columns">
+                    <label>Input Label</label>
+                    <input type="text" placeholder="large-12.columns" />
+                </div>
+            </div>
+            <div class="row">
+                <div class="large-4 medium-4 columns">
+                    <label>Input Label</label>
+                    <input type="text" placeholder="large-4.columns" />
+                </div>
+                <div class="large-4 medium-4 columns">
+                    <label>Input Label</label>
+                    <input type="text" placeholder="large-4.columns" />
+                </div>
+                <div class="large-4 medium-4 columns">
+                    <div class="row collapse">
+                        <label>Input Label</label>
+                        <div class="small-9 columns">
+                            <input type="text" placeholder="small-9.columns" />
+                        </div>
+                        <div class="small-3 columns">
+                            <span class="postfix">.com</span>
+                        </div>
+                    </div>
+                </div>
+            </div>
+            <div class="row">
+                <div class="large-12 columns">
+                    <label>Select Box</label>
+                    <select>
+                        <option value="husker">Husker</option>
+                        <option value="starbuck">Starbuck</option>
+                        <option value="hotdog">Hot Dog</option>
+                        <option value="apollo">Apollo</option>
+                    </select>
+                </div>
+            </div>
+            <div class="row">
+                <div class="large-6 medium-6 columns">
+                    <label>Choose Your Favorite</label>
+                    <input type="radio" name="pokemon" value="Red" id="pokemonRed"><label for="pokemonRed">Radio 1</label>
+                    <input type="radio" name="pokemon" value="Blue" id="pokemonBlue"><label for="pokemonBlue">Radio 2</label>
+                </div>
+                <div class="large-6 medium-6 columns">
+                    <label>Check these out</label>
+                    <input id="checkbox1" type="checkbox"><label for="checkbox1">Checkbox 1</label>
+                    <input id="checkbox2" type="checkbox"><label for="checkbox2">Checkbox 2</label>
+                </div>
+            </div>
+            <div class="row">
+                <div class="large-12 columns">
+                    <label>Textarea Label</label>
+                    <textarea placeholder="small-12.columns"></textarea>
+                </div>
+            </div>
+        </form>
+    </div>     
+
+    <div class="large-4 medium-4 columns">
+        <h5>Try one of these buttons:</h5>
+        <p><a href="#" class="small button">Simple Button</a><br/>
+            <a href="#" class="small radius button">Radius Button</a><br/>
+            <a href="#" class="small round button">Round Button</a><br/>            
+            <a href="#" class="medium success button">Success Btn</a><br/>
+            <a href="#" class="medium alert button">Alert Btn</a><br/>
+            <a href="#" class="medium secondary button">Secondary Btn</a></p>           
+        <div class="panel">
+            <h5>So many components, girl!</h5>
+            <p>A whole kitchen sink of goodies comes with Foundation. Check out the docs to see them all, along with details on making them your own.</p>
+            <a href="http://foundation.zurb.com/docs/" class="small button">Go to Foundation Docs</a>          
+        </div>
+    </div>
+</div>-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/html/mailing_lists.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/html/mailing_lists.hbs b/nifi-site/src/pages/html/mailing_lists.hbs
new file mode 100644
index 0000000..076b684
--- /dev/null
+++ b/nifi-site/src/pages/html/mailing_lists.hbs
@@ -0,0 +1,70 @@
+---
+title: Apache NiFi Mailing Lists
+---
+
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h1 class="nifi-txt">
+            <span>
+                Apache <span class="ni">ni</span><span class="fi">fi</span> Mailing Lists
+            </span>
+        </h1>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <p class="description">
+            These are the mailing lists that have been established for this project. For each list, there is a subscribe, unsubscribe, and an archive link.
+        </p>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <table>
+            <tr>
+                <th>Name</th>
+                <th>Subscribe</th>
+                <th>Unsubscribe</th>
+                <th>Post</th>
+                <th>Archive</th>
+            </tr>
+            <tr>
+                <td>Developers Mailing List</td>
+                <td><a class="externalLink" href="mailto:dev-subscribe@nifi.incubator.apache.org">Subscribe</a></td>
+                <td><a class="externalLink" href="mailto:dev-unsubscribe@nifi.incubator.apache.org">Unsubscribe</a></td>
+                <td><a class="externalLink" href="mailto:dev@nifi.incubator.apache.org">Post</a></td>
+                <td><a class="externalLink" href="http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev/">mail-archives.apache.org</a></td>
+            </tr>
+            <tr>
+                <td>Commits Mailing List</td>
+                <td><a class="externalLink" href="mailto:commits-subscribe@nifi.incubator.apache.org">Subscribe</a></td>
+                <td><a class="externalLink" href="mailto:commits-unsubscribe@nifi.incubator.apache.org">Unsubscribe</a></td>
+                <td><a class="externalLink" href="mailto:commits@nifi.incubator.apache.org">Post</a></td>
+                <td><a class="externalLink" href="http://mail-archives.apache.org/mod_mbox/incubator-nifi-commits/">mail-archives.apache.org</a></td>
+            </tr>
+        </table>
+    </div>
+</div>
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h2 class="nifi-txt">
+            <span>
+                Mailing List Archives
+            </span>
+        </h2>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <p class="description">
+            Some other popular archive sources which are not part of the official Apache archives.
+        </p>
+        <ul>
+            <li>
+                <a href="http://apache-nifi-incubating-developer-list.39713.n7.nabble.com/">Developer List archive on Nabble</a>
+            </li>
+        </ul>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/html/people.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/html/people.hbs b/nifi-site/src/pages/html/people.hbs
new file mode 100644
index 0000000..77b8d9c
--- /dev/null
+++ b/nifi-site/src/pages/html/people.hbs
@@ -0,0 +1,136 @@
+---
+title: Apache NiFi Team
+---
+
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h1 class="nifi-txt">
+            <span>
+                Apache <span class="ni">ni</span><span class="fi">fi</span> Team
+            </span>
+        </h1>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <p class="description">A successful project requires many people to play many roles. Some members write code, provide project mentorship, or author documentation. Others are valuable as testers, submitting patches and suggestions.</p>
+        <p class="description">The team is comprised of Project Management Committee Members and Contributors. Members have direct access to the source of a project and actively evolve the code-base. Contributors improve the project through submission of patches and suggestions to the Members. The number of Contributors to the project is unbounded. Get involved today. All contributions to the project are greatly appreciated.</p>
+    </div>
+</div>
+<div class="medium-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h2 class="nifi-txt">
+            <span>
+                PPMC Members
+            </span>
+        </h2>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <table>
+            <tr>
+                <th>Id</th>
+                <th>Name</th>
+                <th>Affiliation</th>
+            </tr>
+            <tr>
+                <td>apurtell</td>
+                <td>Andrew Purtell</td>
+                <td>Mentor</td>
+            </tr>
+            <tr>
+                <td>arvind</td>
+                <td>Arvind Prabhakar </td>
+                <td>Mentor</td>
+            </tr>
+            <tr>
+                <td>billie</td>
+                <td>Billie Rinaldi</td>
+                <td>Mentor</td>
+            </tr>
+            <tr>
+                <td>bimargulies</td>
+                <td>Benson Margulies</td>
+                <td>Champion</td>
+            </tr>
+            <tr>
+                <td>brock</td>
+                <td>Brock Noland</td>
+                <td>Mentor</td>
+            </tr>
+            <tr>
+                <td>devriesb</td>
+                <td>Brandon DeVries</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>drew</td>
+                <td>Drew Farris</td>
+                <td>Mentor</td>
+            </tr>
+            <tr>
+                <td>jcarey</td>
+                <td>Jason Carey</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>markap14</td>
+                <td>Mark Payne</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>joewitt</td>
+                <td>Joe Witt</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>jbarnabee</td>
+                <td>Jenn Barnabee</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>mcgilman</td>
+                <td>Matt Gilman</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>taftster</td>
+                <td>Adam Taft</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>tkurc</td>
+                <td>Tony Kurc</td>
+                <td></td>
+            </tr>
+            <tr>
+                <td>wikier</td>
+                <td>Sergio Fernandez </td>
+                <td>Mentor</td>
+            </tr>
+        </table>
+    </div>
+</div>
+<div class="medium-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h2 class="nifi-txt">
+            <span>
+                Committers
+            </span>
+        </h2>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <table>
+            <tr>
+                <th>Id</th>
+                <th>Name</th>
+            </tr>
+        </table>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/html/roadmap.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/html/roadmap.hbs b/nifi-site/src/pages/html/roadmap.hbs
new file mode 100644
index 0000000..45da108
--- /dev/null
+++ b/nifi-site/src/pages/html/roadmap.hbs
@@ -0,0 +1,30 @@
+---
+title: Apache NiFi Roadmap
+---
+
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h1 class="nifi-txt">
+            <span>
+                Apache <span class="ni">ni</span><span class="fi">fi</span> Roadmap
+            </span>
+        </h1>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <h3 class="nifi-txt">
+            <span>
+                First steps
+            </span>
+        </h3>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <p class="description">
+            Initial work on Apache NiFi (incubating) will be focused on an initial incubating release. This means, migrating to apache infrastructure, using apache license compatible dependencies, developing developer documentation, and working out bugs introduced or discovered during this process.
+        </p>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/9c61bdb1/nifi-site/src/pages/markdown/quickstart.md
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/markdown/quickstart.md b/nifi-site/src/pages/markdown/quickstart.md
new file mode 100644
index 0000000..14801aa
--- /dev/null
+++ b/nifi-site/src/pages/markdown/quickstart.md
@@ -0,0 +1,95 @@
+---
+title:     Apache NiFi Development Quickstart
+---
+
+This documentation is in progress, but should get many started at building Apache NiFi.
+
+## Source Code
+
+Apache NiFi source code is version controlled using [Git][git] version control ([browse][gitbrowse]|[checkout][gitrepo]).  
+
+The code is also mirrored to [Github][githubrepo]
+
+The code as it was initially contributed and entered the incubator is on the 'master' branch.
+
+To view the lastest codebase as we work toward an initial release checkout the 'develop' branch.
+
+All guidance that follows assumes you are working on the 'develop' branch.
+
+## Issue Tracking
+
+Track issues on the "NIFI" Project on the Apache Jira ([browse][jira]).
+
+## Building
+
+#### Checking out from Git
+
+To check out the code:
+
+    git clone http://git-wip-us.apache.org/repos/asf/incubator-nifi.git
+
+Then checkout the 'develop' branch
+
+    git checkout develop
+
+### Build steps
+
+1. You need a recent Java 7 (or newer) JDK.
+2. You need Apache [Maven 3.X][maven]. We've successfully used 3.2.3 and as far back as 3.0.5
+3. Build the maven plugins.  In the root dir of the source tree cd to `nifi-nar-maven-plugin`.
+   Run `mvn clean install`
+4. Build the entire code base.  In the root dir of the source tree cd to `nifi` and run `mvn -T C2.0 clean install`
+   You can tweak the maven build settings as you like but the previous command will execute with 2 threads per core.
+
+Now you should have a fully functioning build off the latest code in the develop branch.
+
+## Running the application
+
+#### ** WARNING **
+
+Without any configuration, the application will run on port 8080 and does not require any credentials to modify
+the flow. This means of running Apache NiFi should be used only for development/testing and in an environment where only
+connections from trusted computers and users can connect to port 8080. Using iptables to allow only localhost connections
+to 8080 is a good start, but on systems with multiple (potentially untrusted) users, also not a sufficient protection.
+
+#### Decompress and launch
+
+Running the above build will create a tar.gz (and zip) file in `nifi/nifi-assembly/target`. This tar.gz should
+contain the full application. Decompressing the tar.gz should make a directory for you containing several other
+directories. `conf` contains application configuration, `bin` contains scripts
+for launching the application. On linux and OSX, NiFi can be run using `bin/nifi.sh <command>` where
+`<command>` is one of:
+
++ start: starts NiFi in the background
++ stop: stops NiFi that is running in the background
++ status: provides the current status of NiFi
++ run: runs NiFi in the foreground and waits to receive a Ctrl-C, which then shuts down NiFi.
++ install: (available in Linux only, not OSX): installs NiFi as a service that can then be controlled
+via `service nifi start`, `service nifi stop`, `service nifi status`.
+
+
+For Windows users, there exist several scripts in the `bin` directory that are analogous to those above:
+`start-nifi.bat`, `stop-nifi.bat`, `nifi-status.bat`, and `run-nifi.bat`.
+
+The configuration that is to be used when launching NiFi, such as Java heap size, the user
+to run as, which Java command to use, etc. are configurable via the `conf/bootstrap.conf` file.
+
+The entire concept of how the application will integrate to a given OS and run as an
+enduring service is something we're working hard on and would appreciate ideas for.  The user experience needs to
+be excellent.
+
+With the default settings you can point a web browser at `http://localhost:8080/nifi/`
+
+Logging is configured by default to log to `./logs/nifi-app.log`. The following log message should indicate the web ui
+is ready for use:
+
+    2014-12-09 00:42:03,540 INFO [main] org.apache.nifi.web.server.JettyServer NiFi has started. The UI is available at the following URLs:
+
+
+[maven]: http://maven.apache.org/
+[jira]: https://issues.apache.org/jira/browse/NIFI
+[git]: http://git-scm.com/
+[gitbrowse]: https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=summary
+[gitrepo]: http://git-wip-us.apache.org/repos/asf/incubator-nifi.git
+[githubrepo]: https://github.com/apache/incubator-nifi
+


[35/36] incubator-nifi git commit: Merge branch 'develop' into NIFI-250

Posted by mc...@apache.org.
Merge branch 'develop' into NIFI-250

Conflicts:
	nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
	nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
	nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java


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

Branch: refs/heads/NIFI-250
Commit: cc8b096ffa11ee44f6d5fb5f07de51826dd27b20
Parents: 13fb1a7 ed53b46
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 4 12:44:56 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 4 12:44:56 2015 -0500

----------------------------------------------------------------------
 .gitignore                                      |    7 +
 README.md                                       |   14 +
 nifi-nar-maven-plugin/README.md                 |   14 +
 nifi-nar-maven-plugin/pom.xml                   |    8 +-
 nifi-site/Gruntfile.js                          |  137 ++
 nifi-site/LICENSE                               |  202 +++
 nifi-site/README.md                             |  119 ++
 nifi-site/bower.json                            |   19 +
 nifi-site/config.rb                             |   24 +
 nifi-site/package.json                          |   24 +
 nifi-site/robots.txt                            |    4 +
 nifi-site/src/images/bgNifiLogo.png             |  Bin 0 -> 4232 bytes
 nifi-site/src/images/egg-logo.png               |  Bin 0 -> 8626 bytes
 nifi-site/src/images/feather-small.gif          |  Bin 0 -> 7129 bytes
 nifi-site/src/images/flow.png                   |  Bin 0 -> 503302 bytes
 .../src/images/niFi-logo-horizontal-scaled.png  |  Bin 0 -> 4462 bytes
 nifi-site/src/images/niFi-logo-horizontal.png   |  Bin 0 -> 13161 bytes
 nifi-site/src/images/nifi16.ico                 |  Bin 0 -> 1150 bytes
 nifi-site/src/images/nifiDrop.svg               |  217 +++
 nifi-site/src/includes/disclaimer.hbs           |   27 +
 nifi-site/src/includes/footer.hbs               |    6 +
 nifi-site/src/includes/header.hbs               |   22 +
 nifi-site/src/includes/topbar.hbs               |   70 +
 nifi-site/src/js/app.js                         |   15 +
 nifi-site/src/layouts/html.hbs                  |    3 +
 nifi-site/src/layouts/markdown.hbs              |   12 +
 nifi-site/src/pages/html/download.hbs           |   46 +
 nifi-site/src/pages/html/faq.hbs                |   33 +
 nifi-site/src/pages/html/index.hbs              |  210 +++
 nifi-site/src/pages/html/mailing_lists.hbs      |   70 +
 nifi-site/src/pages/html/people.hbs             |  136 ++
 nifi-site/src/pages/html/roadmap.hbs            |   30 +
 nifi-site/src/pages/markdown/quickstart.md      |  101 ++
 nifi-site/src/pages/markdown/release-guide.md   |  298 ++++
 nifi-site/src/scss/_settings.scss               | 1465 ++++++++++++++++++
 nifi-site/src/scss/app.scss                     |  185 +++
 nifi/README.md                                  |   14 +
 nifi/nifi-assembly/pom.xml                      |    1 +
 .../src/main/java/MyProcessor.java              |   10 +-
 .../nifi/util/StandardProcessorTestRunner.java  |    7 +-
 .../util/TestStandardProcessorTestRunner.java   |    2 +
 .../src/main/resources/conf/logback.xml         |    4 +-
 .../nifi/web/controller/ControllerFacade.java   |   15 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |   14 +-
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  117 --
 .../apache/nifi/web/servlet/DownloadSvg.java    |   89 ++
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |   30 +-
 .../src/main/webapp/css/provenance.css          |    5 +
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js |   17 +-
 .../webapp/js/nf/canvas/nf-process-group.js     |    6 +-
 .../js/nf/canvas/nf-remote-process-group.js     |    2 +-
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  319 ++--
 .../webapp/js/nf/counters/nf-counters-table.js  |   63 +-
 .../webapp/js/nf/history/nf-history-table.js    |  149 +-
 .../src/main/webapp/js/nf/nf-common.js          |    7 +-
 .../js/nf/provenance/nf-provenance-lineage.js   |   17 +-
 .../js/nf/provenance/nf-provenance-table.js     |  103 +-
 .../webapp/js/nf/summary/nf-summary-table.js    |  523 +++----
 .../src/main/webapp/js/nf/summary/nf-summary.js |    6 -
 .../js/nf/templates/nf-templates-table.js       |   73 +-
 .../main/webapp/js/nf/users/nf-users-table.js   |  440 +++---
 .../nifi/processors/standard/BinFiles.java      |  351 +++++
 .../nifi/processors/standard/MergeContent.java  |  403 ++---
 .../nifi/processors/standard/PostHTTP.java      |   42 +-
 .../processors/standard/CaptureServlet.java     |   54 +
 .../nifi/processors/standard/TestPostHTTP.java  |  249 +++
 .../nifi/processors/standard/TestServer.java    |   27 +-
 .../update/attributes/api/RuleResource.java     |   16 +-
 nifi/pom.xml                                    |   21 +-
 69 files changed, 5268 insertions(+), 1446 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/cc8b096f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/cc8b096f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/cc8b096f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
index 3d8a619,72fc549..8b3fbda
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js
@@@ -356,74 -433,6 +433,6 @@@ nf.HistoryTable = (function () 
  
              // request refresh of the current 'page'
              historyGrid.onViewportChanged.notify();
-         },
-         
-         /**
-          * Shows the details for the specified action.
-          * 
-          * @param {object} index
-          */
-         showActionDetails: function (index) {
-             var historyGrid = $('#history-table').data('gridInstance');
-             if (nf.Common.isDefinedAndNotNull(historyGrid)) {
-                 var historyModel = historyGrid.getData();
-                 var action = historyModel.getItem(index);
- 
-                 // create the markup for the dialog
-                 var detailsMarkup = $('<div></div>').append(
-                         $('<div class="action-detail"><div class="history-details-name">Id</div>' + nf.Common.escapeHtml(action.sourceId) + '</div>'));
- 
-                 // get any component details
-                 var componentDetails = action.componentDetails;
- 
-                 // inspect the operation to determine if there are any component details
-                 if (nf.Common.isDefinedAndNotNull(componentDetails)) {
-                     if (action.sourceType === 'Processor' || action.sourceType === 'ControllerService') {
-                         detailsMarkup.append(
-                                 $('<div class="action-detail"><div class="history-details-name">Type</div>' + nf.Common.escapeHtml(componentDetails.type) + '</div>'));
-                     } else if (action.sourceType === 'RemoteProcessGroup') {
-                         detailsMarkup.append(
-                                 $('<div class="action-detail"><div class="history-details-name">Uri</div>' + nf.Common.formatValue(componentDetails.uri) + '</div>'));
-                     }
-                 }
- 
-                 // get any action details
-                 var actionDetails = action.actionDetails;
- 
-                 // inspect the operation to determine if there are any action details
-                 if (nf.Common.isDefinedAndNotNull(actionDetails)) {
-                     if (action.operation === 'Configure') {
-                         detailsMarkup.append(
-                                 $('<div class="action-detail"><div class="history-details-name">Name</div>' + nf.Common.formatValue(actionDetails.name) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Value</div>' + nf.Common.formatValue(actionDetails.value) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Previous Value</div>' + nf.Common.formatValue(actionDetails.previousValue) + '</div>'));
-                     } else if (action.operation === 'Connect' || action.operation === 'Disconnect') {
-                         detailsMarkup.append(
-                                 $('<div class="action-detail"><div class="history-details-name">Source Id</div>' + nf.Common.escapeHtml(actionDetails.sourceId) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Source Name</div>' + nf.Common.formatValue(actionDetails.sourceName) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Source Type</div>' + nf.Common.escapeHtml(actionDetails.sourceType) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Relationship(s)</div>' + nf.Common.formatValue(actionDetails.relationship) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Destination Id</div>' + nf.Common.escapeHtml(actionDetails.destinationId) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Destination Name</div>' + nf.Common.formatValue(actionDetails.destinationName) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Destination Type</div>' + nf.Common.escapeHtml(actionDetails.destinationType) + '</div>'));
-                     } else if (action.operation === 'Move') {
-                         detailsMarkup.append(
-                                 $('<div class="action-detail"><div class="history-details-name">Group</div>' + nf.Common.formatValue(actionDetails.group) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Group Id</div>' + nf.Common.escapeHtml(actionDetails.groupId) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Previous Group</div>' + nf.Common.formatValue(actionDetails.previousGroup) + '</div>')).append(
-                                 $('<div class="action-detail"><div class="history-details-name">Previous Group Id</div>' + nf.Common.escapeHtml(actionDetails.previousGroupId) + '</div>'));
-                     } else if (action.operation === 'Purge') {
-                         detailsMarkup.append(
-                                 $('<div class="action-detail"><div class="history-details-name">End Date</div>' + nf.Common.escapeHtml(actionDetails.endDate) + '</div>'));
-                     }
-                 }
- 
-                 // populate the dialog
-                 $('#action-details').append(detailsMarkup);
- 
-                 // show the dialog
-                 $('#action-details-dialog').modal('show');
-             }
          }
      };
--}());
++}());

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/cc8b096f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
index 53ea076,b49a97b..0bd8061
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js
@@@ -407,75 -406,8 +407,74 @@@ nf.Common = 
      }()),
      
      /**
 -     * Creates a form inline in order to post the specified params to the specified URL.
 +     * Determines if the specified property is sensitive.
 +     * 
 +     * @argument {object} propertyDescriptor        The property descriptor
 +     */
 +    isSensitiveProperty: function (propertyDescriptor) {
 +        if (nf.Common.isDefinedAndNotNull(propertyDescriptor)) {
 +            return propertyDescriptor.sensitive === true;
 +        } else {
 +            return false;
 +        }
 +    },
 +
 +    /**
 +     * Determines if the specified property is required.
 +     * 
 +     * @param {object} propertyDescriptor           The property descriptor
 +     */
 +    isRequiredProperty: function (propertyDescriptor) {
 +        if (nf.Common.isDefinedAndNotNull(propertyDescriptor)) {
 +            return propertyDescriptor.required === true;
 +        } else {
 +            return false;
 +        }
 +    },
 +
 +    /**
 +     * Determines if the specified property is required.
 +     * 
 +     * @param {object} propertyDescriptor           The property descriptor
 +     */
 +    isDynamicProperty: function (propertyDescriptor) {
 +        if (nf.Common.isDefinedAndNotNull(propertyDescriptor)) {
 +            return propertyDescriptor.dynamic === true;
 +        } else {
 +            return false;
 +        }
 +    },
 +
 +    /**
 +     * Gets the allowable values for the specified property.
 +     * 
 +     * @argument {object} propertyDescriptor        The property descriptor
 +     */
 +    getAllowableValues: function (propertyDescriptor) {
 +        if (nf.Common.isDefinedAndNotNull(propertyDescriptor)) {
 +            return propertyDescriptor.allowableValues;
 +        } else {
 +            return null;
 +        }
 +    },
 +
 +    /**
 +     * Returns whether the specified property supports EL.
 +     * 
 +     * @param {object} propertyDescriptor           The property descriptor
 +     */
 +    supportsEl: function (propertyDescriptor) {
 +        if (nf.Common.isDefinedAndNotNull(propertyDescriptor)) {
 +            return propertyDescriptor.supportsEl === true;
 +        } else {
 +            return false;
 +        }
 +    },
 +    
 +    /**
 +     * Creates a form inline in order to submit the specified params to the specified URL
 +     * using the specified method.
       * 
-      * @param {string} method       The method to use
       * @param {string} url          The URL
       * @param {object} params       An object with the params to include in the submission
       */
@@@ -962,4 -894,4 +961,4 @@@
          });
          return formattedBulletins;
      }
--};
++};


[22/36] incubator-nifi git commit: NIFI-315: - Moving to a newer version of yui compressor plugin.

Posted by mc...@apache.org.
NIFI-315:
- Moving to a newer version of yui compressor plugin.

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

Branch: refs/heads/NIFI-250
Commit: be16371b20eeb94c90cf369917325c76c3549e1c
Parents: 4b852eb
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Feb 3 09:54:54 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Feb 3 09:54:54 2015 -0500

----------------------------------------------------------------------
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml                    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/be16371b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 202bfc1..9eaa1b9 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -238,7 +238,7 @@
                     <plugin>
                         <groupId>net.alchim31.maven</groupId>
                         <artifactId>yuicompressor-maven-plugin</artifactId>
-                        <version>1.1</version>
+                        <version>1.5.1</version>
                         <executions>
                             <execution>
                                 <id>minify</id>


[28/36] incubator-nifi git commit: NIFI-162: - Updating topbar. - Adding more content.

Posted by mc...@apache.org.
NIFI-162:
- Updating topbar.
- Adding more content.

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

Branch: refs/heads/NIFI-250
Commit: 8140b2a9475f32e131d2302b74d9864694c06ab5
Parents: be16371
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Feb 3 22:44:07 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Feb 3 22:44:07 2015 -0500

----------------------------------------------------------------------
 nifi-site/src/includes/topbar.hbs             |   7 +-
 nifi-site/src/layouts/markdown.hbs            |   1 +
 nifi-site/src/pages/html/download.hbs         |  46 +++++
 nifi-site/src/pages/markdown/quickstart.md    |  14 +-
 nifi-site/src/pages/markdown/release-guide.md | 208 +++++++++++----------
 nifi-site/src/scss/_settings.scss             |  11 +-
 nifi-site/src/scss/app.scss                   |   8 +
 7 files changed, 190 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/includes/topbar.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/includes/topbar.hbs b/nifi-site/src/includes/topbar.hbs
index bbf7821..e368385 100644
--- a/nifi-site/src/includes/topbar.hbs
+++ b/nifi-site/src/includes/topbar.hbs
@@ -20,7 +20,6 @@
                     <ul class="dropdown">
                         <li><a href="index.html">Home</a></li>
                         <li><a href="https://blogs.apache.org/nifi/"><i class="fa fa-external-link external-link"></i>Apache NiFi Blog</a></li>
-                        <li><a href="#">Downloads</a></li>
                         <li><a href="roadmap.html">Roadmap</a></li>
                         <li><a href="http://www.apache.org/licenses/LICENSE-2.0"><i class="fa fa-external-link external-link"></i>License</a></li>
                     </ul>
@@ -35,6 +34,12 @@
                     </ul>
                 </li>
                 <li class="has-dropdown">
+                    <a href="#">Download</a>
+                    <ul class="dropdown">
+                        <li><a href="download.html">Download NiFi</a></li>
+                    </ul>
+                </li>
+                <li class="has-dropdown">
                     <a href="#">Community</a>
                     <ul class="dropdown">
                         <li><a href="mailing_lists.html">Mailing Lists</a></li>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/layouts/markdown.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/layouts/markdown.hbs b/nifi-site/src/layouts/markdown.hbs
index fdfda90..a68c5a8 100644
--- a/nifi-site/src/layouts/markdown.hbs
+++ b/nifi-site/src/layouts/markdown.hbs
@@ -2,6 +2,7 @@
 
 <div class="row">
     <div class="large-12 columns">
+        <div class="large-space"></div>
         {{#markdown}}
             {{> body }}
         {{/markdown}}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/pages/html/download.hbs
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/html/download.hbs b/nifi-site/src/pages/html/download.hbs
new file mode 100644
index 0000000..379d0e8
--- /dev/null
+++ b/nifi-site/src/pages/html/download.hbs
@@ -0,0 +1,46 @@
+---
+title: Apache NiFi FAQs
+---
+
+<div class="large-space"></div>
+<div class="row">
+    <div class="large-12 columns">
+        <h1 class="nifi-txt">
+            <span>
+                Apache <span class="ni">ni</span><span class="fi">fi</span> Downloads
+            </span>
+        </h1>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <p>To verify the downloads please follow these <a href="http://www.apache.org/info/verification.html">procedures</a>
+            using these <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS">KEYS</a></p>
+        <p>If a download is not found please allow up to 24 hours for the mirrors to sync.</p>
+    </div>
+</div>
+<div class="row">
+    <div class="large-12 columns">
+        <h2>Releases</h2>
+        <ul>
+            <li>0.0.1-incubating
+                <ul>
+                    <li><a href="https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&amp;version=12329078">Release Notes</a></li>
+                    <li>
+                        Sources:
+                        <ul>
+                            <li><a href="https://www.apache.org/dyn/closer.cgi?path=/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip">nifi-0.0.1-incubating-source-release.zip</a> ( <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip.asc">asc</a>, <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip.md5">md5</a>, <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip.sha1">sha1</a> )</li>
+                        </ul>
+                    </li>
+                    <li>
+                        Binaries
+                        <ul>
+                            <li><a href="https://www.apache.org/dyn/closer.cgi?path=/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.tar.gz">nifi-0.0.1-incubating-bin.tar.gz</a> ( <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.tar.gz.asc">asc</a>, <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.tar.gz.md5">md5</a>, <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.tar.gz.sha1">sha1</a> )</li>
+                            <li><a href="https://www.apache.org/dyn/closer.cgi?path=/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.zip">nifi-0.0.1-incubating-bin.zip</a> ( <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.zip.asc">asc</a>, <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.zip.md5">md5</a>, <a href="https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating/nifi-0.0.1-incubating-bin.zip.sha1">sha1</a> )</li>
+                        </ul>
+                    </li>
+                </ul>
+            </li>
+        </ul>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/pages/markdown/quickstart.md
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/markdown/quickstart.md b/nifi-site/src/pages/markdown/quickstart.md
index 14801aa..c0148d7 100644
--- a/nifi-site/src/pages/markdown/quickstart.md
+++ b/nifi-site/src/pages/markdown/quickstart.md
@@ -2,6 +2,8 @@
 title:     Apache NiFi Development Quickstart
 ---
 
+# Apache NiFi Development Quickstart
+
 This documentation is in progress, but should get many started at building Apache NiFi.
 
 ## Source Code
@@ -26,12 +28,16 @@ Track issues on the "NIFI" Project on the Apache Jira ([browse][jira]).
 
 To check out the code:
 
-    git clone http://git-wip-us.apache.org/repos/asf/incubator-nifi.git
-
+```
+git clone http://git-wip-us.apache.org/repos/asf/incubator-nifi.git
+```
+<br/>
 Then checkout the 'develop' branch
 
-    git checkout develop
-
+```
+git checkout develop
+```
+<br/>
 ### Build steps
 
 1. You need a recent Java 7 (or newer) JDK.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/pages/markdown/release-guide.md
----------------------------------------------------------------------
diff --git a/nifi-site/src/pages/markdown/release-guide.md b/nifi-site/src/pages/markdown/release-guide.md
index e7750c1..7387692 100644
--- a/nifi-site/src/pages/markdown/release-guide.md
+++ b/nifi-site/src/pages/markdown/release-guide.md
@@ -2,6 +2,8 @@
 title: Apache NiFi Release Guidelines
 ---
 
+# Apache NiFi Release Guidelines
+
 The purpose of this document is to capture and describe the steps involved in producing 
 an official release of Apache NiFi.  It is written specifically to someone acting in the
 capacity of a [Release Manager][release-manager] (RM).  
@@ -82,9 +84,11 @@ There are two lists here: one of specific incubator requirements, and another of
 ### Prepare your environment
   
 Follow the steps outlined in the [Quickstart Guide][quickstart-guide]
-        
-    At this point you're on the latest 'develop' branch and are able to build the entire application
 
+```        
+At this point you're on the latest 'develop' branch and are able to build the entire application
+```
+<br/>
 Create a JIRA ticket for the release tasks and use that ticket number for the commit messages.  For example we'll consider NIFI-270 as our ticket.  Also
 have in mind the release version you are planning for.  For example we'll consider '0.0.1-incubating'.
 
@@ -101,24 +105,26 @@ work `MAVEN_OPTS="-Xms1024m -Xmx3076m -XX:MaxPermSize=256m"`
 
 Ensure your settings.xml has been updated as shown below.  There are other ways to ensure your PGP key is available for signing as well
   
->          ...
->          <profile>
->             <id>signed_release</id>
->             <properties>
->                 <mavenExecutorId>forked-path</mavenExecutorId>
->                 <gpg.keyname>YOUR GPG KEY ID HERE</gpg.keyname>
->                 <gpg.passphrase>YOUR GPG PASSPHRASE HERE</gpg.passphrase>
->             </properties>
->         </profile>
->         ...
->         <servers>
->            <server>
->                <id>repository.apache.org</id>
->                <username>YOUR USER NAME HERE</username>
->                <password>YOUR MAVEN ENCRYPTED PASSWORD HERE</password>
->            </server>
->         </servers>
->         ...
+```
+        ...
+        <profile>
+           <id>signed_release</id>
+           <properties>
+               <mavenExecutorId>forked-path</mavenExecutorId>
+               <gpg.keyname>YOUR GPG KEY ID HERE</gpg.keyname>
+               <gpg.passphrase>YOUR GPG PASSPHRASE HERE</gpg.passphrase>
+           </properties>
+       </profile>
+       ...
+       <servers>
+          <server>
+              <id>repository.apache.org</id>
+              <username>YOUR USER NAME HERE</username>
+              <password>YOUR MAVEN ENCRYPTED PASSWORD HERE</password>
+          </server>
+       </servers>
+       ...
+```
 
 Ensure the the full application build and tests all work by executing
 `mvn -T 2.5C clean install` for a parallel build.  Once that completes you can
@@ -177,94 +183,100 @@ commands can come in handy to help do this so you can set versions to something
 Now it's time to initiate a vote within the PPMC.  Send the vote request to `dev@nifi.incubator.apache.org`
 with a subject of `[VOTE] Release Apache NiFi nifi-nar-maven-plugin-0.0.1-incubating`. The following template can be used:
  
->     Hello
->     I am pleased to be calling this vote for the source release of Apache NiFi
->     nifi-nar-maven-plugin-0.0.1-incubating.
->     
->     The source zip, including signatures, digests, etc. can be found at:
->     https://repository.apache.org/content/repositories/orgapachenifi-1011
->     
->     The Git tag is nifi-nar-maven-plugin-0.0.1-incubating-RC1
->     The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
->     https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
->     
->     Checksums of nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip:
->     MD5: 5a580756a17b0573efa3070c70585698
->     SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
->     
->     Release artifacts are signed with the following key:
->     https://people.apache.org/keys/committer/joewitt.asc
->     
->     KEYS file available here:
->     https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
->     
->     8 issues were closed/resolved for this release:
->     https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
->     
->     The vote will be open for 72 hours. 
->     Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test.  The please vote:
->     
->     [ ] +1 Release this package as nifi-nar-maven-plugin-0.0.1-incubating
->     [ ] +0 no opinion
->     [ ] -1 Do not release this package because because...
+```
+Hello
+I am pleased to be calling this vote for the source release of Apache NiFi
+nifi-nar-maven-plugin-0.0.1-incubating.
+
+The source zip, including signatures, digests, etc. can be found at:
+https://repository.apache.org/content/repositories/orgapachenifi-1011
+
+The Git tag is nifi-nar-maven-plugin-0.0.1-incubating-RC1
+The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
+https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
+
+Checksums of nifi-nar-maven-plugin-0.0.1-incubating-source-release.zip:
+MD5: 5a580756a17b0573efa3070c70585698
+SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
+
+Release artifacts are signed with the following key:
+https://people.apache.org/keys/committer/joewitt.asc
+
+KEYS file available here:
+https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
 
+8 issues were closed/resolved for this release:
+https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
+
+The vote will be open for 72 hours. 
+Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test.  The please vote:
+
+[ ] +1 Release this package as nifi-nar-maven-plugin-0.0.1-incubating
+[ ] +0 no opinion
+[ ] -1 Do not release this package because because...
+```
+<br/>
 A release vote is majority rule.  So wait 72 hours and see if there are at least 3 binding +1 votes and no more negative votes than positive.
 If so forward the vote to the IPMC.  Send the vote request to `general@incubator.apache.org` with a subject of
 `[VOTE] Release Apache NiFi nifi-nar-maven-plugin-0.0.1-incubating`.  The following template can be used:
 
->     Hello
->     
->     The Apache NiFi PPMC has voted to release Apache NiFi nar-maven-plugin-0.0.1-incubating.
->     The vote was based on the release candidate and thread described below.
->     We now request the IPMC to vote on this release.
->     
->     Here is the PPMC voting result:
->     X +1 (binding)
->     Y -1 (binding)
->     
->     Here is the PPMC vote thread: [URL TO PPMC Vote Thread]
->     
->     The source zip, including signatures, digests, etc. can be found at:
->     https://repository.apache.org/content/repositories/orgapachenifi-1011
->     
->     The Git tag is nar-maven-plugin-0.0.1-incubating-RC1
->     The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
->     https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
->     
->     Checksums of nar-maven-plugin-0.0.1-incubating-source-release.zip:
->     MD5: 5a580756a17b0573efa3070c70585698
->     SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
->     
->     Release artifacts are signed with the following key:
->     https://people.apache.org/keys/committer/joewitt.asc
->     
->     KEYS file available here:
->     https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
->     
->     8 issues were closed/resolved for this release:
->     https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
->     
->     The vote will be open for 72 hours. 
->     Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test.  The please vote:
->     
->     [ ] +1 Release this package as nar-maven-plugin-0.0.1-incubating
->     [ ] +0 no opinion
->     [ ] -1 Do not release this package because because...
+```
+Hello
+
+The Apache NiFi PPMC has voted to release Apache NiFi nar-maven-plugin-0.0.1-incubating.
+The vote was based on the release candidate and thread described below.
+We now request the IPMC to vote on this release.
+
+Here is the PPMC voting result:
+X +1 (binding)
+Y -1 (binding)
+
+Here is the PPMC vote thread: [URL TO PPMC Vote Thread]
 
+The source zip, including signatures, digests, etc. can be found at:
+https://repository.apache.org/content/repositories/orgapachenifi-1011
+
+The Git tag is nar-maven-plugin-0.0.1-incubating-RC1
+The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
+https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
+
+Checksums of nar-maven-plugin-0.0.1-incubating-source-release.zip:
+MD5: 5a580756a17b0573efa3070c70585698
+SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
+
+Release artifacts are signed with the following key:
+https://people.apache.org/keys/committer/joewitt.asc
+
+KEYS file available here:
+https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
+
+8 issues were closed/resolved for this release:
+https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
+
+The vote will be open for 72 hours. 
+Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test.  The please vote:
+
+[ ] +1 Release this package as nar-maven-plugin-0.0.1-incubating
+[ ] +0 no opinion
+[ ] -1 Do not release this package because because...
+```
+<br/>
 Wait 72 hours.  If the vote passes then send a vote result email.  Send the email to `general@incubator.apache.org, dev@nifi.incubator.apache.org`
 with a subject of `[RESULT][VOTE] Release Apache NiFi nar-maven-plugin-0.0.1-incubating`.  Use a template such as:
 
->     Hello
->     
->     The release passes with
->     
->     X +1 (binding) votes
->     Y -1 (binding) votes
->     
->     Thanks to all who helped make this release possible.
->     
->     Here is the IPMC vote thread: [INSERT URL OF IPMC Vote Thread]
+```
+Hello
+
+The release passes with
+
+X +1 (binding) votes
+Y -1 (binding) votes
+
+Thanks to all who helped make this release possible.
 
+Here is the IPMC vote thread: [INSERT URL OF IPMC Vote Thread]
+```
+<br/>
 Now all the voting is done and the release is good to go.  In repository.apache.org go to the staging repository
 and select `release`.  Then publish the source, hashes, and signatures to `https://dist.apache.org/repos/dist/release/incubator/nifi/`
 Then merge the release git tag to develop and to master.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/scss/_settings.scss
----------------------------------------------------------------------
diff --git a/nifi-site/src/scss/_settings.scss b/nifi-site/src/scss/_settings.scss
index ff2d464..e66a214 100644
--- a/nifi-site/src/scss/_settings.scss
+++ b/nifi-site/src/scss/_settings.scss
@@ -196,6 +196,7 @@ $font-weight-bold: bold !default;
 
 // We use these to control header font styles
 // $header-font-family: $body-font-family;
+$header-font-family: 'Oswald', sans-serif;
 // $header-font-weight: $font-weight-normal;
 // $header-font-style: normal;
 // $header-font-color: $jet;
@@ -251,14 +252,20 @@ $font-weight-bold: bold !default;
 // $code-border-style: solid;
 // $code-border-color: scale-color($code-background-color, $lightness: -10%);
 // $code-padding: rem-calc(2) rem-calc(5) rem-calc(1);
+/*$code-color: scale-color($alert-color, $lightness: -27%);*/
+$code-color: #333;
+$code-font-family: Consolas, Liberation Mono, Courier, monospace;
+$code-font-weight: bold;
+$code-border-size: 0px;
+$code-font-weight: normal;
 
 // We use these to style anchors
 // $anchor-text-decoration: none;
 // $anchor-text-decoration-hover: none;
 // $anchor-font-color: $primary-color;
-$anchor-font-color: #1e373f;
+$anchor-font-color: #396877;
 // $anchor-font-color-hover: scale-color($primary-color, $lightness: -14%);
-$anchor-font-color: #264c58;
+$anchor-font-color-hover: #264c58;
 
 // We use these to style the <hr> element
 // $hr-border-width: 1px;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8140b2a9/nifi-site/src/scss/app.scss
----------------------------------------------------------------------
diff --git a/nifi-site/src/scss/app.scss b/nifi-site/src/scss/app.scss
index 935ef74..869a7fb 100644
--- a/nifi-site/src/scss/app.scss
+++ b/nifi-site/src/scss/app.scss
@@ -52,6 +52,14 @@ html, html a {
     visibility:hidden;
 }
 
+pre code {
+    display: block;
+    overflow: auto;
+    padding: 10px;
+    font-weight: normal;
+    border-width: 1px;
+}
+
 div.large-space {
     height: 3em;
 }


[36/36] incubator-nifi git commit: NIFI-250: - Fixing use of the optimistic locking manager factory bean.

Posted by mc...@apache.org.
NIFI-250:
- Fixing use of the optimistic locking manager factory bean.

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

Branch: refs/heads/NIFI-250
Commit: 6b91546d9d04dcb127cada3cdc907e9ef98abf42
Parents: cc8b096
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 4 13:13:09 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 4 13:13:09 2015 -0500

----------------------------------------------------------------------
 .../nifi/web/spring/OptimisticLockingManagerFactoryBean.java   | 6 ------
 .../nifi-web-api/src/main/resources/nifi-web-api-context.xml   | 4 +++-
 2 files changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/6b91546d/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java
index 850d2fd..8112b7b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java
@@ -16,16 +16,10 @@
  */
 package org.apache.nifi.web.spring;
 
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.controller.FlowController;
-import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.ClusterAwareOptimisticLockingManager;
-import org.apache.nifi.web.ClusterRequestException;
 import org.apache.nifi.web.OptimisticLockingManager;
 import org.apache.nifi.web.StandardOptimisticLockingManager;
-import org.apache.nifi.web.dao.ControllerServiceDAO;
-import org.apache.nifi.web.dao.impl.StandardControllerServiceDAO;
 import org.springframework.beans.BeansException;
 import org.springframework.beans.factory.FactoryBean;
 import org.springframework.context.ApplicationContext;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/6b91546d/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 61b68c5..94887c7 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -25,7 +25,9 @@
     http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
 
     <!-- optimistic locking manager -->
-    <bean id="webOptimisticLockingManager" class="org.apache.nifi.web.spring.OptimisticLockingManagerFactoryBean" depends-on="clusterManagerOptimisticLockingManager"/>
+    <bean id="webOptimisticLockingManager" class="org.apache.nifi.web.spring.OptimisticLockingManagerFactoryBean" depends-on="clusterManagerOptimisticLockingManager">
+        <property name="properties" ref="nifiProperties"/>
+    </bean>
 
     <!-- dto factory -->
     <bean id="dtoFactory" class="org.apache.nifi.web.api.dto.DtoFactory">