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/01/14 19:35:25 UTC

[01/17] incubator-nifi git commit: NIFI-241: Only call Processor methods with @OnAdded annotation when a Processor is actually added to graph instead of also calling on NiFi restart

Repository: incubator-nifi
Updated Branches:
  refs/heads/NIFI-250 03d422e4d -> bb108a096


NIFI-241: Only call Processor methods with @OnAdded annotation when a Processor is actually added to graph instead of also calling on NiFi restart


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

Branch: refs/heads/NIFI-250
Commit: 21e809a7cbb85f450749916e474dec93f0d023d6
Parents: f36eea3
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 13 13:28:20 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 13 13:28:20 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/controller/FlowController.java  | 35 ++++++++++++++++----
 .../controller/StandardFlowSynchronizer.java    |  2 +-
 .../nifi/fingerprint/FingerprintFactory.java    |  2 +-
 3 files changed, 30 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/21e809a7/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
index 99d8d6e..346e801 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -756,7 +756,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
 
     /**
      * <p>
-     * Creates a new ProcessorNode with the given type and identifier.</p>
+     * Creates a new ProcessorNode with the given type and identifier and initializes it invoking the
+     * methods annotated with {@link OnAdded}.
+     * </p>
      *
      * @param type
      * @param id
@@ -766,6 +768,24 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
      * instantiated for any reason
      */
     public ProcessorNode createProcessor(final String type, String id) throws ProcessorInstantiationException {
+        return createProcessor(type, id, true);
+    }
+    
+    /**
+     * <p>
+     * Creates a new ProcessorNode with the given type and identifier and optionally initializes it.
+     * </p>
+     *
+     * @param type the fully qualified Processor class name
+     * @param id the unique ID of the Processor
+     * @param firstTimeAdded whether or not this is the first time this Processor is added to the graph. If {@code true},
+     *                       will invoke methods annotated with the {@link OnAdded} annotation.
+     * @return
+     * @throws NullPointerException if either arg is null
+     * @throws ProcessorInstantiationException if the processor cannot be
+     * instantiated for any reason
+     */
+    public ProcessorNode createProcessor(final String type, String id, final boolean firstTimeAdded) throws ProcessorInstantiationException {
         id = id.intern();
         final Processor processor = instantiateProcessor(type, id);
         final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider);
@@ -774,12 +794,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
         final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
         logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ProcessorLogObserver(getBulletinRepository(), procNode));
 
-        // TODO: We should only call this the first time that it is added to the graph....
-        try (final NarCloseable x = NarCloseable.withNarLoader()) {
-            ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor);
-        } catch (final Exception e) {
-            logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID);
-            throw new ProcessorLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e);
+        if ( firstTimeAdded ) {
+            try (final NarCloseable x = NarCloseable.withNarLoader()) {
+                ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor);
+            } catch (final Exception e) {
+                logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID);
+                throw new ProcessorLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e);
+            }
         }
 
         return procNode;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/21e809a7/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index ffea644..b60d187 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -594,7 +594,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         final List<Element> processorNodeList = getChildrenByTagName(processGroupElement, "processor");
         for (final Element processorElement : processorNodeList) {
             final ProcessorDTO processorDTO = FlowFromDOMFactory.getProcessor(processorElement, encryptor);
-            final ProcessorNode procNode = controller.createProcessor(processorDTO.getType(), processorDTO.getId());
+            final ProcessorNode procNode = controller.createProcessor(processorDTO.getType(), processorDTO.getId(), false);
             processGroup.addProcessor(procNode);
             updateProcessor(procNode, processorDTO, processGroup, controller);
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/21e809a7/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 9be46a1..7bdf278 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -467,7 +467,7 @@ public final class FingerprintFactory {
         Processor processor = null;
         try {
             if (controller != null) {
-                processor = controller.createProcessor(className, UUID.randomUUID().toString()).getProcessor();
+                processor = controller.createProcessor(className, UUID.randomUUID().toString(), false).getProcessor();
             }
         } catch (ProcessorInstantiationException e) {
             logger.warn("Unable to create Processor of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", className, e.toString());


[16/17] incubator-nifi git commit: NIFI-250: - Creating endpoints for returning the available controller services and reporting tasks. - Made the Setting tabbed to provide a place for configuring controller services and reporting tasks.

Posted by mc...@apache.org.
NIFI-250:
- Creating endpoints for returning the available controller services and reporting tasks.
- Made the Setting tabbed to provide a place for configuring controller services and reporting tasks.


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

Branch: refs/heads/NIFI-250
Commit: da18ce0ab0fee2e497e65ff4af30dd8dac212e96
Parents: 7a95835
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Jan 13 14:52:23 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Jan 14 13:33:51 2015 -0500

----------------------------------------------------------------------
 .../web/api/dto/DocumentedChildTypeDTO.java     |  43 ++++
 .../entity/ControllerServiceTypesEntity.java    |  46 ++++
 .../api/entity/ReportingTaskTypesEntity.java    |  46 ++++
 .../org/apache/nifi/web/NiFiServiceFacade.java  |  16 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |  12 +-
 .../apache/nifi/web/api/ControllerResource.java |  68 +++++
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  50 ++++
 .../nifi/web/controller/ControllerFacade.java   |  22 ++
 .../partials/canvas/settings-content.jsp        |  82 ++++---
 .../src/main/webapp/css/settings.css            |  84 ++++++-
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   |   1 +
 .../src/main/webapp/js/nf/canvas/nf-settings.js | 246 +++++++++++++------
 12 files changed, 603 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedChildTypeDTO.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedChildTypeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedChildTypeDTO.java
new file mode 100644
index 0000000..f34798a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedChildTypeDTO.java
@@ -0,0 +1,43 @@
+/*
+ * 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.api.dto;
+
+import java.util.Set;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * Class used for providing documentation of a specified child type that may be
+ * instantiated. Also supports specifying what the underlying base classes are.
+ */
+@XmlType(name = "documentedChildType")
+public class DocumentedChildTypeDTO extends DocumentedTypeDTO {
+
+    private Set<String> baseTypes;
+
+    /**
+     * The type is the fully-qualified name of a Java class.
+     *
+     * @return
+     */
+    public Set<String> getBaseType() {
+        return baseTypes;
+    }
+
+    public void setBaseType(Set<String> baseTypes) {
+        this.baseTypes = baseTypes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java
new file mode 100644
index 0000000..622d1dc
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import java.util.Set;
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of
+ * a response to the API. This particular entity holds a reference to a list of
+ * controller service types.
+ */
+@XmlRootElement(name = "controllerServiceTypesEntity")
+public class ControllerServiceTypesEntity extends Entity {
+
+    private Set<DocumentedTypeDTO> controllerServiceTypes;
+
+    /**
+     * The list of controller service types that are being serialized.
+     *
+     * @return
+     */
+    public Set<DocumentedTypeDTO> getControllerSerivceTypes() {
+        return controllerServiceTypes;
+    }
+
+    public void setControllerServiceTypes(Set<DocumentedTypeDTO> controllerServiceTypes) {
+        this.controllerServiceTypes = controllerServiceTypes;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java
new file mode 100644
index 0000000..4b021ef
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import java.util.Set;
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of
+ * a response to the API. This particular entity holds a reference to a list of
+ * reporting task types.
+ */
+@XmlRootElement(name = "reportingTaskTypesEntity")
+public class ReportingTaskTypesEntity extends Entity {
+
+    private Set<DocumentedTypeDTO> reportingTaskTypes;
+
+    /**
+     * The list of reporting task types that are being serialized.
+     *
+     * @return
+     */
+    public Set<DocumentedTypeDTO> getReportingTaskTypes() {
+        return reportingTaskTypes;
+    }
+
+    public void setReportingTaskTypes(Set<DocumentedTypeDTO> reportingTaskTypes) {
+        this.reportingTaskTypes = reportingTaskTypes;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index ae6bf28..b0001ec 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -243,7 +243,21 @@ public interface NiFiServiceFacade {
      * @return The list of available processor types
      */
     Set<DocumentedTypeDTO> getProcessorTypes();
-
+    
+    /**
+     * Returns the list of controller service types.
+     * 
+     * @return The list of available controller types
+     */
+    Set<DocumentedTypeDTO> getControllerServiceTypes();
+    
+    /**
+     * Returns the list of reporting task types.
+     * 
+     * @return The list of available reporting task types
+     */
+    Set<DocumentedTypeDTO> getReportingTaskTypes();
+    
     /**
      * Returns the list of prioritizer types.
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index c8683b0..41660f2 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -16,8 +16,6 @@
  */
 package org.apache.nifi.web;
 
-import org.apache.nifi.web.OptimisticLockingManager;
-import org.apache.nifi.web.ConfigurationSnapshot;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -1640,6 +1638,16 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
+    public Set<DocumentedTypeDTO> getControllerServiceTypes() {
+        return controllerFacade.getControllerServiceTypes();
+    }
+
+    @Override
+    public Set<DocumentedTypeDTO> getReportingTaskTypes() {
+        return controllerFacade.getReportingTaskTypes();
+    }
+
+    @Override
     public ProcessorDTO getProcessor(String groupId, String id) {
         final ProcessorNode processor = processorDAO.getProcessor(groupId, id);
         final ProcessorDTO processorDto = dtoFactory.createProcessorDto(processor);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 3afe0e1..d13e4ce 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -72,6 +72,8 @@ import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.IntegerParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity;
+import org.apache.nifi.web.api.entity.ReportingTaskTypesEntity;
 import org.codehaus.enunciate.jaxrs.TypeHint;
 import org.springframework.security.access.prepost.PreAuthorize;
 
@@ -713,6 +715,72 @@ public class ControllerResource extends ApplicationResource {
         // generate the response
         return clusterContext(generateOkResponse(entity)).build();
     }
+    
+    /**
+     * Retrieves the types of controller services that this NiFi supports.
+     *
+     * @param clientId Optional client id. If the client id is not specified, a
+     * new one will be generated. This value (whether specified or generated) is
+     * included in the response.
+     * @return A controllerServicesTypesEntity.
+     */
+    @GET
+    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Path("/controller-service-types")
+    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @TypeHint(ControllerServiceTypesEntity.class)
+    public Response getControllerServiceTypes(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
+
+        // replicate if cluster manager
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
+        }
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // create response entity
+        final ControllerServiceTypesEntity entity = new ControllerServiceTypesEntity();
+        entity.setRevision(revision);
+        entity.setControllerServiceTypes(serviceFacade.getControllerServiceTypes());
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
+    
+    /**
+     * Retrieves the types of reporting tasks that this NiFi supports.
+     *
+     * @param clientId Optional client id. If the client id is not specified, a
+     * new one will be generated. This value (whether specified or generated) is
+     * included in the response.
+     * @return A controllerServicesTypesEntity.
+     */
+    @GET
+    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Path("/reporting-task-types")
+    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @TypeHint(ReportingTaskTypesEntity.class)
+    public Response getReportingTaskTypes(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
+
+        // replicate if cluster manager
+        if (properties.isClusterManager()) {
+            return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
+        }
+
+        // create the revision
+        final RevisionDTO revision = new RevisionDTO();
+        revision.setClientId(clientId.getClientId());
+
+        // create response entity
+        final ReportingTaskTypesEntity entity = new ReportingTaskTypesEntity();
+        entity.setRevision(revision);
+        entity.setReportingTaskTypes(serviceFacade.getControllerServiceTypes());
+
+        // generate the response
+        return clusterContext(generateOkResponse(entity)).build();
+    }
 
     /**
      * Retrieves the types of prioritizers that this NiFi supports.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 9a2dc30..9ef6287 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -1128,6 +1128,56 @@ public final class DtoFactory {
 
         return types;
     }
+    
+    /**
+     * Identifies all baseTypes for the specified type that are assignable to the specified baseType.
+     * 
+     * @param baseType
+     * @param type
+     * @param baseTypes 
+     */
+    private void identifyBaseTypes(final Class baseType, final Class type, final Set<String> baseTypes) {
+        final Class[] interfaces = type.getInterfaces();
+        for (final Class i : interfaces) {
+            if (baseType.isAssignableFrom(i) && !baseType.equals(i)) {
+                baseTypes.add(i.getName());
+            }
+        }
+        
+        if (type.getSuperclass() != null) {
+            identifyBaseTypes(baseType, type.getSuperclass(), baseTypes);
+        }
+    }
+    
+    /**
+     * Gets the DocumentedTypeDTOs from the specified classes for the specified baseClass.
+     *
+     * @param baseClass
+     * @param classes
+     * @return
+     */
+    public Set<DocumentedTypeDTO> fromDocumentedTypes(final Class baseClass, final Set<Class> classes) {
+        final Set<DocumentedTypeDTO> types = new LinkedHashSet<>();
+        final Set<Class> sortedClasses = new TreeSet<>(CLASS_NAME_COMPARATOR);
+        sortedClasses.addAll(classes);
+
+        for (final Class<?> cls : sortedClasses) {
+            final DocumentedChildTypeDTO type = new DocumentedChildTypeDTO();
+            type.setType(cls.getName());
+            type.setDescription(getCapabilityDescription(cls));
+            type.setTags(getTags(cls));
+            
+            // identify the base types
+            final Set<String> baseTypes = new LinkedHashSet<>();
+            identifyBaseTypes(baseClass, cls, baseTypes);
+            type.setBaseType(baseTypes);
+            
+            // add this type
+            types.add(type);
+        }
+
+        return types;
+    }
 
     /**
      * Creates a ProcessorDTO from the specified ProcessorNode.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index dbc4b3c..a9ad968 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -114,6 +114,8 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.authorization.DownloadAuthorization;
 import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.reporting.ReportingTask;
+import org.apache.nifi.web.api.dto.DocumentedChildTypeDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.security.access.AccessDeniedException;
@@ -347,6 +349,24 @@ public class ControllerFacade implements ControllerServiceProvider {
     public Set<DocumentedTypeDTO> getFlowFileComparatorTypes() {
         return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(FlowFilePrioritizer.class));
     }
+    
+    /**
+     * Gets the ControllerService types that this controller supports.
+     * 
+     * @return 
+     */
+    public Set<DocumentedTypeDTO> getControllerServiceTypes() {
+        return dtoFactory.fromDocumentedTypes(ControllerService.class, ExtensionManager.getExtensions(ControllerService.class));
+    }
+    
+    /**
+     * Gets the ReportingTask types that this controller supports.
+     * 
+     * @return 
+     */
+    public Set<DocumentedTypeDTO> getReportingTaskTypes() {
+        return dtoFactory.fromDocumentedTypes(ReportingTask.class, ExtensionManager.getExtensions(ReportingTask.class));
+    }
 
     /**
      * Gets the counters for this controller.
@@ -371,6 +391,8 @@ public class ControllerFacade implements ControllerServiceProvider {
 
         return counter;
     }
+    
+    
 
     /**
      * Return the controller service for the specified identifier.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
index d9b2f29..b235bdf 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp
@@ -18,43 +18,59 @@
 <div id="settings">
     <div id="settings-header-text">NiFi Settings</div>
     <div id="settings-container">
-        <div id="general-settings">
-            <div class="setting">
-                <div class="setting-name">Data flow name</div>
-                <div class="setting-field">
-                    <input type="text" id="data-flow-title-field" name="data-flow-title" class="setting-input"/>
-                    <span id="archive-flow-link" class="link">Back-up flow</span>
-                    <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Archives the flow configuration."/>
-                </div>
-            </div>
-            <div class="setting">
-                <div class="setting-name">Data flow comments</div>
-                <div class="setting-field">
-                    <textarea id="data-flow-comments-field" name="data-flow-comments" class="setting-input"></textarea>
+        <div id="settings-tabs-container">
+            <div id="settings-tabs"></div>
+            <div id="new-service-or-task" class="add-icon-bg"></div>
+            <div class="clear"></div>
+        </div>
+        <div id="settings-tab-background"></div>
+        <div id="settings-tabs-content">
+            <div id="general-settings-tab-content" class="configuration-tab">
+                <div id="general-settings">
+                    <div class="setting">
+                        <div class="setting-name">Data flow name</div>
+                        <div class="setting-field">
+                            <input type="text" id="data-flow-title-field" name="data-flow-title" class="setting-input"/>
+                            <span id="archive-flow-link" class="link">Back-up flow</span>
+                            <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="Archives the flow configuration."/>
+                        </div>
+                    </div>
+                    <div class="setting">
+                        <div class="setting-name">Data flow comments</div>
+                        <div class="setting-field">
+                            <textarea id="data-flow-comments-field" name="data-flow-comments" class="setting-input"></textarea>
+                        </div>
+                    </div>
+                    <div class="setting">
+                        <div class="setting-name">
+                            Maximum timer driven thread count
+                            <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="The maximum number of threads for timer driven processors available to the system."/>
+                        </div>
+                        <div class="setting-field">
+                            <input type="text" id="maximum-timer-driven-thread-count-field" class="setting-input"/>
+                        </div>
+                    </div>
+                    <div class="setting">
+                        <div class="setting-name">
+                            Maximum event driven thread count
+                            <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="The maximum number of threads for event driven processors available to the system."/>
+                        </div>
+                        <div class="setting-field">
+                            <input type="text" id="maximum-event-driven-thread-count-field" class="setting-input"/>
+                        </div>
+                    </div>
+                    <div id="settings-buttons">
+                        <div id="settings-save" class="button">Apply</div>
+                        <div class="clear"></div>
+                    </div>
                 </div>
             </div>
-            <div class="setting">
-                <div class="setting-name">
-                    Maximum timer driven thread count
-                    <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="The maximum number of threads for timer driven processors available to the system."/>
-                </div>
-                <div class="setting-field">
-                    <input type="text" id="maximum-timer-driven-thread-count-field" class="setting-input"/>
-                </div>
+            <div id="controller-services-tab-content" class="configuration-tab">
+                <div id="controller-services-table" class="settings-table"></div>
             </div>
-            <div class="setting">
-                <div class="setting-name">
-                    Maximum event driven thread count
-                    <img class="setting-icon icon-info" src="images/iconInfo.png" alt="Info" title="The maximum number of threads for event driven processors available to the system."/>
-                </div>
-                <div class="setting-field">
-                    <input type="text" id="maximum-event-driven-thread-count-field" class="setting-input"/>
-                </div>
+            <div id="reporting-tasks-tab-content" class="configuration-tab">
+                <div id="reporting-tasks-table" class="settings-table"></div>
             </div>
         </div>
-        <div id="settings-buttons">
-            <div id="settings-cancel" class="button">Cancel</div>
-            <div id="settings-save" class="button">Apply</div>
-        </div>
     </div>
 </div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/css/settings.css
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/css/settings.css b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/css/settings.css
index e9d78d4..0622aa9 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/css/settings.css
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/css/settings.css
@@ -21,23 +21,93 @@
     bottom: 0px;
     left: 0px;
     display: none;
+    padding: 20px;
 }
 
 #settings-header-text {
     height: 35px;
-    margin-top: 20px;
-    margin-left: 20px;
     font-size: 16px;
     font-weight: bold;
 }
 
 #settings-container {
-    margin-top: -10px;
+    margin-top: 18px;
 }
 
+#new-service-or-task {
+    float: right;
+    width: 19px;
+    height: 19px;
+    margin-top: 4px;
+    cursor: pointer;
+}
+
+/* settings tabs */
+
+#settings-tabs-container {
+    border-bottom: 2px solid #666;
+}
+
+#settings-tabs {
+    float: left;
+}
+
+.settings-tab {
+    display: block;
+    padding: 0 5px;
+    height: 26px;
+    float: left;
+    color: #666;
+    background-color: #ccc;
+    border-left: 1px solid #b5b5b5;
+    border-top: 1px solid #b5b5b5;
+    border-right: 1px solid #b5b5b5;
+    margin-right: 5px;
+    text-align: center;
+    cursor: pointer;
+    line-height: 26px;
+    font-weight: bold;
+}
+
+.settings-selected-tab {
+    color: #fff;
+    background-color: #666;
+    border-left: 1px solid #666;
+    border-top: 1px solid #666;
+    border-right: 1px solid #666;
+}
+
+#settings-tab-background {
+    height: 200px;
+    margin-top: 1px;
+    background-color: transparent;
+    background: linear-gradient(to bottom, #dddddd, #ffffff);
+    filter: progid:DXImageTransform.Microsoft.gradient(gradientType=0, startColorstr=#dddddd, endColorstr=#ffffff);
+}
+
+#settings div.configuration-tab {
+    display: none;
+}
+
+div.settings-table {
+    position: absolute;
+    top: 115px;
+    left: 25px;
+    bottom: 20px;
+    right: 25px;
+    border: 1px solid #666;
+    overflow: hidden;
+    background-color: #fff;
+}
+
+span.sorted {
+    text-decoration: underline;
+}
+
+/* general */
+
 #general-settings {
-    padding-left: 10px;
-    margin-top: 15px;
+    margin-top: -190px;
     margin-left: 10px;
 }
 
@@ -53,11 +123,11 @@
 }
 
 #settings-buttons {
-    margin-left: 260px;
+    margin-left: 304px;
     margin-top: 10px;
 }
 
-#settings-save, #settings-cancel {
+#settings-save {
     float: left;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
index 26db837..8125219 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
@@ -509,6 +509,7 @@ nf.Canvas = (function () {
         // listen for browser resize events to reset the graph size
         $(window).on('resize', function () {
             updateGraphSize();
+            nf.Settings.resetTableSize();
         }).on('keydown', function (evt) {
             // if a dialog is open, disable canvas shortcuts
             if ($('.dialog').is(':visible')) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/da18ce0a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
index 3d573b3..8015be5 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
@@ -19,11 +19,75 @@ nf.Settings = (function () {
     var config = {
         urls: {
             controllerConfig: '../nifi-api/controller/config',
-            controllerArchive: '../nifi-api/controller/archive'
+            controllerArchive: '../nifi-api/controller/archive',
+            controllerServiceTypes: '../nifi-api/controller/controller-service-types',
+            reportingTaskTypes: '../nifi-api/controller/reporting-task-types'
         }
     };
 
     /**
+     * Initializes the general tab.
+     */
+    var initGeneral = function () {
+        // register the click listener for the archive link
+        $('#archive-flow-link').click(function () {
+            var revision = nf.Client.getRevision();
+
+            $.ajax({
+                type: 'POST',
+                url: config.urls.controllerArchive,
+                data: {
+                    version: revision.version,
+                    clientId: revision.clientId
+                },
+                dataType: 'json'
+            }).done(function (response) {
+                // update the revision
+                nf.Client.setRevision(response.revision);
+
+                // show the result dialog
+                nf.Dialog.showOkDialog({
+                    dialogContent: 'A new flow archive was successfully created.',
+                    overlayBackground: false
+                });
+            }).fail(nf.Common.handleAjaxError);
+        });
+
+        // register the click listener for the save button
+        $('#settings-save').click(function () {
+            var revision = nf.Client.getRevision();
+
+            // marshal the configuration details
+            var configuration = marshalConfiguration();
+            configuration['version'] = revision.version;
+            configuration['clientId'] = revision.clientId;
+
+            // save the new configuration details
+            $.ajax({
+                type: 'PUT',
+                url: config.urls.controllerConfig,
+                data: configuration,
+                dataType: 'json'
+            }).done(function (response) {
+                // update the revision
+                nf.Client.setRevision(response.revision);
+
+                // update the displayed name
+                document.title = response.config.name;
+
+                // set the data flow title and close the shell
+                $('#data-flow-title-container').children('span.link:first-child').text(response.config.name);
+
+                // close the settings dialog
+                nf.Dialog.showOkDialog({
+                    dialogContent: 'Settings successfully applied.',
+                    overlayBackground: false
+                });
+            }).fail(nf.Common.handleAjaxError);
+        });
+    };
+
+    /**
      * Marshals the details to include in the configuration request.
      */
     var marshalConfiguration = function () {
@@ -35,83 +99,125 @@ nf.Settings = (function () {
         configuration['maxEventDrivenThreadCount'] = $('#maximum-event-driven-thread-count-field').val();
         return configuration;
     };
+    
+    
+    
+    /**
+     * Initializes the controller services tab.
+     */
+    var initControllerServies = function () {
+        $.ajax({
+            type: 'GET',
+            url: config.urls.controllerServiceTypes,
+            dataType: 'json'
+        }).done(function(response) {
+            console.log(response);
+        });
+        
+        var moreControllerServiceDetails = function (row, cell, value, columnDef, dataContext) {
+            return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 5px; float: left;" onclick="javascript:nf.Settings.showControllerServiceDetails(\'' + row + '\');"/>';
+        };
+        
+        // define the column model for the controller services table
+        var controllerServicesColumnModel = [
+            {id: 'moreDetails', field: 'moreDetails', name: '&nbsp;', resizable: false, formatter: moreControllerServiceDetails, sortable: true, width: 50, maxWidth: 50},
+            {id: 'id', field: 'id', name: 'Identifier', sortable: true, resizable: true},
+            {id: 'type', field: 'type', name: 'Type', sortable: true, resizable: true}
+        ];
+    };
+    
+    /**
+     * Initializes the reporting tasks tab.
+     */
+    var initReportingTasks = function () {
+        $.ajax({
+            type: 'GET',
+            url: config.urls.reportingTaskTypes,
+            dataType: 'json'
+        }).done(function(response) {
+            console.log(response);
+        });
+        
+        var moreReportingTaskDetails = function (row, cell, value, columnDef, dataContext) {
+            return '<img src="images/iconDetails.png" title="View Details" class="pointer" style="margin-top: 5px; float: left;" onclick="javascript:nf.Settings.showControllerServiceDetails(\'' + row + '\');"/>';
+        };
+        
+        // define the column model for the reporting tasks table
+        var reportingTasksColumnModel = [
+            {id: 'moreDetails', field: 'moreDetails', name: '&nbsp;', resizable: false, formatter: moreReportingTaskDetails, sortable: true, width: 50, maxWidth: 50},
+            {id: 'id', field: 'id', name: 'Identifier', sortable: true, resizable: true},
+            {id: 'type', field: 'type', name: 'Type', sortable: true, resizable: true},
+            
+        ];
+    };
 
     return {
         /**
          * Initializes the status page.
          */
         init: function () {
-
-            // register the click listener for the archive link
-            $('#archive-flow-link').click(function () {
-                var revision = nf.Client.getRevision();
-
-                $.ajax({
-                    type: 'POST',
-                    url: config.urls.controllerArchive,
-                    data: {
-                        version: revision.version,
-                        clientId: revision.clientId
-                    },
-                    dataType: 'json'
-                }).done(function (response) {
-                    // update the revision
-                    nf.Client.setRevision(response.revision);
-
-                    // show the result dialog
-                    nf.Dialog.showOkDialog({
-                        dialogContent: 'A new flow archive was successfully created.',
-                        overlayBackground: false
-                    });
-                }).fail(function (xhr, status, error) {
-                    // close the details panel
-                    $('#settings-cancel').click();
-
-                    // handle the error
-                    nf.Common.handleAjaxError(xhr, status, error);
-                });
+            // initialize the settings tabs
+            $('#settings-tabs').tabbs({
+                tabStyle: 'settings-tab',
+                selectedTabStyle: 'settings-selected-tab',
+                tabs: [{
+                    name: 'General',
+                    tabContentId: 'general-settings-tab-content'
+                }, {
+                    name: 'Controller Services',
+                    tabContentId: 'controller-services-tab-content'
+                }, {
+                    name: 'Reporting Tasks',
+                    tabContentId: 'reporting-tasks-tab-content'
+                }],
+                select: function () {
+                    var tab = $(this).text();
+                    if (tab === 'General') {
+                        $('#new-service-or-task').hide();
+                    } else {
+                        $('#new-service-or-task').show();
+                        
+                        // update the tooltip on the button
+                        $('#new-service-or-task').attr('title', function() {
+                            if (tab === 'Controller Services') {
+                                return 'Create a new controller service';
+                            } else if (tab === 'Reporting Tasks') {
+                                return 'Create a new reporting task';
+                            }
+                        });
+                    }
+                }
             });
-
-            // register the click listener for the save button
-            $('#settings-save').click(function () {
-                var revision = nf.Client.getRevision();
-
-                // marshal the configuration details
-                var configuration = marshalConfiguration();
-                configuration['version'] = revision.version;
-                configuration['clientId'] = revision.clientId;
-
-                // save the new configuration details
-                $.ajax({
-                    type: 'PUT',
-                    url: config.urls.controllerConfig,
-                    data: configuration,
-                    dataType: 'json'
-                }).done(function (response) {
-                    // update the revision
-                    nf.Client.setRevision(response.revision);
-
-                    // update the displayed name
-                    document.title = response.config.name;
-
-                    // set the data flow title and close the shell
-                    $('#data-flow-title-container').children('span.link:first-child').text(response.config.name);
-
-                    // close the settings dialog
-                    $('#shell-close-button').click();
-                }).fail(function (xhr, status, error) {
-                    // close the details panel
-                    $('#settings-cancel').click();
-
-                    // handle the error
-                    nf.Common.handleAjaxError(xhr, status, error);
-                });
+            
+            // create a new controller service or reporting task
+            $('#new-service-or-task').on('click', function() {
+                var selectedTab = $('li.settings-selected-tab').text();
+                if (selectedTab === 'Controller Services') {
+                    
+                } else if (selectedTab === 'Reporting Tasks') {
+                    
+                }
             });
 
-            // install a cancel button listener to close the shell
-            $('#settings-cancel').click(function () {
-                $('#shell-close-button').click();
-            });
+            // initialize each tab
+            initGeneral();
+            initControllerServies();
+            initReportingTasks();
+        },
+        
+        /**
+         * Update the size of the grid based on its container's current size.
+         */
+        resetTableSize: function () {
+            var controllerServicesGrid = $('#controller-services-table').data('gridInstance');
+            if (nf.Common.isDefinedAndNotNull(controllerServicesGrid)) {
+                controllerServicesGrid.resizeCanvas();
+            }
+
+            var reportingTasksGrid = $('#reporting-tasks-table').data('gridInstance');
+            if (nf.Common.isDefinedAndNotNull(reportingTasksGrid)) {
+                reportingTasksGrid.resizeCanvas();
+            }
         },
         
         /**
@@ -138,7 +244,7 @@ nf.Settings = (function () {
                 // show the settings dialog
                 nf.Shell.showContent('#settings').done(function () {
                     // reset button state
-                    $('#settings-save, #settings-cancel').mouseout();
+                    $('#settings-save').mouseout();
                 });
             }).fail(nf.Common.handleAjaxError);
         }


[06/17] incubator-nifi git commit: NIFI-220: Added error handling that was missing for one instance of calling producer.send, also indicated how many messages were sent per FlowFile in log message and provenance event

Posted by mc...@apache.org.
NIFI-220: Added error handling that was missing for one instance of calling producer.send, also indicated how many messages were sent per FlowFile in log message and provenance event


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

Branch: refs/heads/NIFI-250
Commit: a77fb50116642c9692e18046ba42663ea8240087
Parents: 95b22a0
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 13 19:27:07 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 13 19:27:07 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/processors/kafka/PutKafka.java  | 28 +++++++++++---------
 1 file changed, 16 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a77fb501/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index 4b5a742..51f9ef1 100644
--- a/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -98,15 +98,15 @@ public class PutKafka extends AbstractProcessor {
 		.defaultValue(DELIVERY_BEST_EFFORT.getValue())
 		.build();
     public static final PropertyDescriptor MESSAGE_DELIMITER = new PropertyDescriptor.Builder()
-            .name("Message Delimiter")
-            .description("Specifies the delimiter to use for splitting apart multiple messages within a single FlowFile. "
-                    + "If not specified, the entire content of the FlowFile will be used as a single message. "
-                    + "If specified, the contents of the FlowFile will be split on this delimiter and each section "
-                    + "sent as a separate Kafka message.")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Message Delimiter")
+        .description("Specifies the delimiter to use for splitting apart multiple messages within a single FlowFile. "
+                + "If not specified, the entire content of the FlowFile will be used as a single message. "
+                + "If specified, the contents of the FlowFile will be split on this delimiter and each section "
+                + "sent as a separate Kafka message.")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
         .name("Max Buffer Size")
         .description("The maximum amount of data to buffer in memory before sending to Kafka")
@@ -366,7 +366,11 @@ public class PutKafka extends AbstractProcessor {
 
                             // If there are messages left, send them
                             if ( !messages.isEmpty() ) {
-                                producer.send(messages);
+                                try {
+                                    producer.send(messages);
+                                } catch (final Exception e) {
+                                    throw new ProcessException("Failed to send messages to Kafka", e);
+                                }
                             }
                         }
                     }
@@ -374,9 +378,9 @@ public class PutKafka extends AbstractProcessor {
                 
                 final long nanos = System.nanoTime() - start;
                 
-                session.getProvenanceReporter().send(flowFile, "kafka://" + topic);
+                session.getProvenanceReporter().send(flowFile, "kafka://" + topic, "Sent " + messagesSent.get() + " messages");
                 session.transfer(flowFile, REL_SUCCESS);
-                getLogger().info("Successfully sent {} to Kafka in {} millis", new Object[] {flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)});
+                getLogger().info("Successfully sent {} messages to Kafka for {} in {} millis", new Object[] {messagesSent.get(), flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)});
             } catch (final ProcessException pe) {
                 error = true;
                 


[13/17] incubator-nifi git commit: NIFI-264: Make getIdentifier method of AbstractSessionFactoryProcessor final so that subclasses cannot override it

Posted by mc...@apache.org.
NIFI-264: Make getIdentifier method of AbstractSessionFactoryProcessor final so that subclasses cannot override it


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

Branch: refs/heads/NIFI-250
Commit: d3aec885145ab46aa79812f613db8a3762adb443
Parents: 7737fbd
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jan 14 12:26:43 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jan 14 12:26:43 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/processor/AbstractSessionFactoryProcessor.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d3aec885/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java b/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java
index 6f1059f..8ccd145 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java
@@ -110,7 +110,7 @@ public abstract class AbstractSessionFactoryProcessor extends AbstractConfigurab
     }
 
     @Override
-    public String getIdentifier() {
+    public final String getIdentifier() {
         return identifier;
     }
 


[02/17] 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/95b22a0a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/95b22a0a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/95b22a0a

Branch: refs/heads/NIFI-250
Commit: 95b22a0aee87f29a5ebcfacd6a5fb274ca95b837
Parents: 21e809a b6b1859
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 13 13:35:39 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 13 13:35:39 2015 -0500

----------------------------------------------------------------------
 .../nifi-web-ui/src/main/webapp/css/summary.css |  2 ++
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 21 ++++++++++++++++----
 2 files changed, 19 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[07/17] incubator-nifi git commit: NIFI-220: Added to provenance event number of events received

Posted by mc...@apache.org.
NIFI-220: Added to provenance event number of events received


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

Branch: refs/heads/NIFI-250
Commit: eaca6a29d3a33fcbe466c8aff87cc8ad3060b9e3
Parents: a77fb50
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 13 19:28:53 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 13 19:28:53 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/eaca6a29/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
----------------------------------------------------------------------
diff --git a/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
index 8c6ca4c..f202e29 100644
--- a/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
+++ b/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java
@@ -302,7 +302,7 @@ public class GetKafka extends AbstractProcessor {
     		} else {
         		flowFile = session.putAllAttributes(flowFile, attributes);
         		final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-        		session.getProvenanceReporter().receive(flowFile, "kafka://" + topic, millis);
+        		session.getProvenanceReporter().receive(flowFile, "kafka://" + topic, "Received " + numMessages + " Kafka messages", millis);
         		getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[] {flowFile, numMessages, millis});
         		session.transfer(flowFile, REL_SUCCESS);
     		}


[11/17] 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/c62aba13
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c62aba13
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c62aba13

Branch: refs/heads/NIFI-250
Commit: c62aba1336f0a77d9baae8ba261877d959bf97bb
Parents: 210a21c 5fe4c15
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jan 14 12:22:10 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jan 14 12:22:10 2015 -0500

----------------------------------------------------------------------
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  4 ++
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    |  4 ++
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 12 ++++--
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  8 ++++
 .../webapp/js/nf/canvas/nf-graph-control.js     | 40 ++++++++++++++++----
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  3 ++
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |  4 ++
 7 files changed, 64 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[04/17] incubator-nifi git commit: NIFI-257: - Using the target URI as specified.

Posted by mc...@apache.org.
NIFI-257:
- Using the target URI as specified.

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

Branch: refs/heads/NIFI-250
Commit: b142d7a9bdecee96bcc7c1b920b3c2dbb748b5e9
Parents: 549c97d
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Jan 13 15:21:42 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Jan 13 15:21:42 2015 -0500

----------------------------------------------------------------------
 .../web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b142d7a9/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
index dc8ca31..5fc94e1 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
@@ -167,7 +167,7 @@ nf.Actions = (function () {
                 var uri = selectionData.component.targetUri;
 
                 if (!nf.Common.isBlank(uri)) {
-                    window.open(encodeURI(uri + '/nifi'));
+                    window.open(encodeURI(uri));
                 } else {
                     nf.Dialog.showOkDialog({
                         dialogContent: 'No target URI defined.'


[05/17] incubator-nifi git commit: NIFI-258: - Ensuring connection stays up to date when processor configuration changes.

Posted by mc...@apache.org.
NIFI-258:
- Ensuring connection stays up to date when processor configuration changes.

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

Branch: refs/heads/NIFI-250
Commit: 80793ccbb829e079ef8cc2fe703bad6397a71d48
Parents: b142d7a
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Jan 13 15:54:21 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Jan 13 15:54:21 2015 -0500

----------------------------------------------------------------------
 .../js/nf/canvas/nf-processor-configuration.js  | 23 +++++++++++++++++++-
 1 file changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/80793ccb/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js
index 4dec734..14ffa95 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js
@@ -363,6 +363,20 @@ nf.ProcessorConfiguration = (function () {
             return true;
         }
     };
+    
+    /**
+     * Reloads the outgoing connections for the specified processor.
+     * 
+     * @param {object} processor
+     */
+    var reloadProcessorConnections = function (processor) {
+        var connections = nf.Connection.getComponentConnections(processor.id);
+        $.each(connections, function (_, connection) {
+            if (connection.source.id === processor.id) {
+                nf.Connection.reload(connection);
+            }
+        });
+    };
 
     return {
         /**
@@ -601,8 +615,11 @@ nf.ProcessorConfiguration = (function () {
                                             // update the revision
                                             nf.Client.setRevision(response.revision);
 
-                                            // set the new processor state
+                                            // set the new processor state based on the response
                                             nf.Processor.set(response.processor);
+                                            
+                                            // reload the processor's outgoing connections
+                                            reloadProcessorConnections(processor);
 
                                             // close the details panel
                                             $('#processor-configuration').modal('hide');
@@ -632,7 +649,11 @@ nf.ProcessorConfiguration = (function () {
 
                                     // show the custom ui
                                     nf.CustomProcessorUi.showCustomUi($('#processor-id').text(), processor.config.customUiUrl, true).done(function () {
+                                        // once the custom ui is closed, reload the processor
                                         nf.Processor.reload(processor);
+                                        
+                                        // and reload the processor's outgoing connections
+                                        reloadProcessorConnections(processor);
                                     });
                                 };
 


[10/17] incubator-nifi git commit: NIFI-261: Sort processor properties before fingerprinting for templates

Posted by mc...@apache.org.
NIFI-261: Sort processor properties before fingerprinting for templates


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

Branch: refs/heads/NIFI-250
Commit: 210a21cd8f13fd477a595dc6c83cffce0b6be3e0
Parents: 67dee2e
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jan 14 12:19:50 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jan 14 12:19:50 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/fingerprint/FingerprintFactory.java   | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/210a21cd/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 7bdf278..8575569 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -29,6 +29,8 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
 import java.util.UUID;
 
 import javax.xml.XMLConstants;
@@ -57,7 +59,6 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
-
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -510,7 +511,8 @@ public final class FingerprintFactory {
         if (config.getProperties() == null) {
             builder.append("NO_PROPERTIES");
         } else {
-            for (final Map.Entry<String, String> entry : config.getProperties().entrySet()) {
+            final SortedMap<String, String> sortedProps = new TreeMap<>(config.getProperties());
+            for (final Map.Entry<String, String> entry : sortedProps.entrySet()) {
                 final String propName = entry.getKey();
                 final String propValue = entry.getValue();
                 if (propValue == null) {


[08/17] 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/67dee2e3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/67dee2e3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/67dee2e3

Branch: refs/heads/NIFI-250
Commit: 67dee2e33b19589ea6756b031ecf68540008bd33
Parents: eaca6a2 80793cc
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jan 14 07:07:20 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jan 14 07:07:20 2015 -0500

----------------------------------------------------------------------
 .../src/main/webapp/js/nf/canvas/nf-actions.js  |  2 +-
 .../js/nf/canvas/nf-processor-configuration.js  | 23 +++++++++++++++++++-
 .../src/main/webapp/js/nf/nf-shell.js           | 10 +++++++++
 3 files changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[03/17] incubator-nifi git commit: NIFI-256: - Hiding the context menu when opening the shell if possible.

Posted by mc...@apache.org.
NIFI-256:
- Hiding the context menu when opening the shell if possible.

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

Branch: refs/heads/NIFI-250
Commit: 549c97d3f4f69a2f04e877bc7b74095fd4a5b0d3
Parents: 95b22a0
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Jan 13 15:17:58 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Jan 13 15:17:58 2015 -0500

----------------------------------------------------------------------
 .../web/nifi-web-ui/src/main/webapp/js/nf/nf-shell.js     | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/549c97d3/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/nf-shell.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/nf-shell.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/nf-shell.js
index b0793a9..85ca0bf 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/nf-shell.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/nf-shell.js
@@ -53,6 +53,11 @@ nf.Shell = (function () {
          * @argument {boolean} canUndock        Whether or not the shell is undockable
          */
         showPage: function (uri, canUndock) {
+            // if the context menu is on this page, attempt to close
+            if (nf.Common.isDefinedAndNotNull(nf.ContextMenu)) {
+                nf.ContextMenu.hide();
+            }
+            
             return $.Deferred(function (deferred) {
                 var shell = $('#shell');
 
@@ -108,6 +113,11 @@ nf.Shell = (function () {
          * @argument {string} domId             The id of the element to show in the shell
          */
         showContent: function (domId) {
+            // if the context menu is on this page, attempt to close
+            if (nf.Common.isDefinedAndNotNull(nf.ContextMenu)) {
+                nf.ContextMenu.hide();
+            }
+            
             return $.Deferred(function (deferred) {
                 var content = $(domId);
                 if (content.length) {


[09/17] incubator-nifi git commit: NIFI-256: - Identifying more places where we should be manually hiding the context menu.

Posted by mc...@apache.org.
NIFI-256:
- Identifying more places where we should be manually hiding the context menu.

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

Branch: refs/heads/NIFI-250
Commit: 5fe4c15cd004015f6b18a2d4a5eb3d0cd9b02f7a
Parents: 67dee2e
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Jan 14 11:33:58 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Jan 14 11:33:58 2015 -0500

----------------------------------------------------------------------
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  4 ++
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    |  4 ++
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 12 ++++--
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  8 ++++
 .../webapp/js/nf/canvas/nf-graph-control.js     | 40 ++++++++++++++++----
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  3 ++
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |  4 ++
 7 files changed, 64 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-birdseye.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-birdseye.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-birdseye.js
index 27e013e..b317a6f 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-birdseye.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-birdseye.js
@@ -272,6 +272,10 @@ nf.Birdseye = (function () {
                             y: d.y
                         };
                     })
+                    .on('dragstart', function () {
+                        // hide the context menu
+                        nf.ContextMenu.hide();
+                    })
                     .on('drag', function (d) {
                         d.x += d3.event.dx;
                         d.y += d3.event.dy;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-toolbox.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-toolbox.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-toolbox.js
index 43a5a17..1d9ba16 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-toolbox.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-toolbox.js
@@ -71,6 +71,10 @@ nf.CanvasToolbox = (function () {
                 return $('<div class="toolbox-icon"></div>').addClass(dragCls).appendTo('body');
             },
             'containment': 'body',
+            'start': function(e, ui) {
+                // hide the context menu if necessary
+                nf.ContextMenu.hide();
+            },
             'stop': function (e, ui) {
                 var translate = nf.Canvas.View.translate();
                 var scale = nf.Canvas.View.scale();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
index 5c1cbb8..26db837 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
@@ -427,9 +427,6 @@ nf.Canvas = (function () {
             // reset the canvas click flag
             canvasClicked = false;
 
-            // hide the context menu if necessary
-            nf.ContextMenu.hide();
-
             // get the selection box 
             var selectionBox = d3.select('rect.selection');
             if (!selectionBox.empty()) {
@@ -866,6 +863,9 @@ nf.Canvas = (function () {
          */
         reload: function () {
             return $.Deferred(function (deferred) {
+                // hide the context menu
+                nf.ContextMenu.hide();
+                
                 // get the process group to refresh everything
                 var processGroupXhr = reloadProcessGroup(nf.Canvas.getGroupId());
                 var statusXhr = reloadFlowStatus();
@@ -983,6 +983,7 @@ nf.Canvas = (function () {
                         // initialize the application
                         initCanvas();
                         nf.Canvas.View.init();
+                        nf.ContextMenu.init();
                         nf.CanvasHeader.init();
                         nf.CanvasToolbox.init();
                         nf.CanvasToolbar.init();
@@ -1077,6 +1078,7 @@ nf.Canvas = (function () {
         setGroupId: function (gi) {
             groupId = gi;
         },
+        
         /**
          * Get the group id.
          */
@@ -1222,6 +1224,10 @@ nf.Canvas = (function () {
                             .scaleExtent([MIN_SCALE, MAX_SCALE])
                             .translate(TRANSLATE)
                             .scale(SCALE)
+                            .on('zoomstart', function () {
+                                // hide the context menu
+                                nf.ContextMenu.hide();
+                            })
                             .on('zoom', function () {
                                 // if we have zoomed, indicate that we are panning
                                 // to prevent deselection elsewhere

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
index 6365dd5..dfb8d7a 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
@@ -382,6 +382,14 @@ nf.ContextMenu = (function () {
     };
 
     return {
+        init: function () {
+            $('#context-menu').on('contextmenu', function(evt) {
+                // stop propagation and prevent default
+                evt.preventDefault();
+                evt.stopPropagation();
+            });
+        },
+        
         /**
          * Shows the context menu. 
          */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-graph-control.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-graph-control.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-graph-control.js
index 852fe72..6537344 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-graph-control.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-graph-control.js
@@ -26,10 +26,13 @@ nf.GraphControl = (function () {
          */
         init: function () {
             // pan up
-            nf.Common.addHoverEffect('#pan-up-button', 'pan-up', 'pan-up-hover').click(function () {
+            nf.Common.addHoverEffect('#pan-up-button', 'pan-up', 'pan-up-hover').on('click', function () {
                 var translate = nf.Canvas.View.translate();
                 nf.Canvas.View.translate([translate[0], translate[1] + config.translateIncrement]);
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -37,10 +40,13 @@ nf.GraphControl = (function () {
             });
 
             // pan down
-            nf.Common.addHoverEffect('#pan-down-button', 'pan-down', 'pan-down-hover').click(function () {
+            nf.Common.addHoverEffect('#pan-down-button', 'pan-down', 'pan-down-hover').on('click', function () {
                 var translate = nf.Canvas.View.translate();
                 nf.Canvas.View.translate([translate[0], translate[1] - config.translateIncrement]);
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -48,10 +54,13 @@ nf.GraphControl = (function () {
             });
 
             // pan left
-            nf.Common.addHoverEffect('#pan-left-button', 'pan-left', 'pan-left-hover').click(function () {
+            nf.Common.addHoverEffect('#pan-left-button', 'pan-left', 'pan-left-hover').on('click', function () {
                 var translate = nf.Canvas.View.translate();
                 nf.Canvas.View.translate([translate[0] + config.translateIncrement, translate[1]]);
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -59,10 +68,13 @@ nf.GraphControl = (function () {
             });
 
             // pan right
-            nf.Common.addHoverEffect('#pan-right-button', 'pan-right', 'pan-right-hover').click(function () {
+            nf.Common.addHoverEffect('#pan-right-button', 'pan-right', 'pan-right-hover').on('click', function () {
                 var translate = nf.Canvas.View.translate();
                 nf.Canvas.View.translate([translate[0] - config.translateIncrement, translate[1]]);
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -70,9 +82,12 @@ nf.GraphControl = (function () {
             });
 
             // zoom in
-            nf.Common.addHoverEffect('#zoom-in-button', 'zoom-in', 'zoom-in-hover').click(function () {
+            nf.Common.addHoverEffect('#zoom-in-button', 'zoom-in', 'zoom-in-hover').on('click', function () {
                 nf.Canvas.View.zoomIn();
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -80,9 +95,12 @@ nf.GraphControl = (function () {
             });
 
             // zoom out
-            nf.Common.addHoverEffect('#zoom-out-button', 'zoom-out', 'zoom-out-hover').click(function () {
+            nf.Common.addHoverEffect('#zoom-out-button', 'zoom-out', 'zoom-out-hover').on('click', function () {
                 nf.Canvas.View.zoomOut();
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -90,9 +108,12 @@ nf.GraphControl = (function () {
             });
 
             // zoom fit
-            nf.Common.addHoverEffect('#zoom-fit-button', 'fit-image', 'fit-image-hover').click(function () {
+            nf.Common.addHoverEffect('#zoom-fit-button', 'fit-image', 'fit-image-hover').on('click', function () {
                 nf.Canvas.View.fit();
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true
@@ -100,9 +121,12 @@ nf.GraphControl = (function () {
             });
 
             // one to one
-            nf.Common.addHoverEffect('#zoom-actual-button', 'actual-size', 'actual-size-hover').click(function () {
+            nf.Common.addHoverEffect('#zoom-actual-button', 'actual-size', 'actual-size-hover').on('click', function () {
                 nf.Canvas.View.actualSize();
 
+                // hide the context menu
+                nf.ContextMenu.hide();
+
                 // refresh the canvas
                 nf.Canvas.View.refresh({
                     transition: true

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-search.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-search.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-search.js
index 2016721..2f7529e 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-search.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-search.js
@@ -166,6 +166,9 @@ nf.Search = (function () {
                     $('div.search-glass-pane').remove();
                 }
             }).focus(function () {
+                // hide the context menu if necessary
+                nf.ContextMenu.hide();
+                
                 // clear the text for the user to type
                 $(this).val('').removeClass('search-flow');
             }).blur(function () {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5fe4c15c/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-toolbar-action.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-toolbar-action.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-toolbar-action.js
index 67a6cc1..d2cde9a 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-toolbar-action.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-toolbar-action.js
@@ -62,6 +62,10 @@ nf.ToolbarAction.prototype.initAction = function () {
         }
     }).click(function () {
         if (!$(this).hasClass(self.disableCls)) {
+            // hide the context menu
+            nf.ContextMenu.hide();
+            
+            // execute the action
             nf.Actions[self.action](nf.CanvasUtils.getSelection());
         }
     }).appendTo(this.container);


[14/17] incubator-nifi git commit: NIFI-260 Replaced four screenshots so they have new logo

Posted by mc...@apache.org.
NIFI-260 Replaced four screenshots so they have new logo

Signed-off-by: Matt Gilman <ma...@gmail.com>


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

Branch: refs/heads/NIFI-250
Commit: a6f0948af1fb12dfbc7f961f5078cd27e12f65cd
Parents: c62aba1
Author: Jenn Barnabee <je...@gmail.com>
Authored: Wed Jan 14 09:10:57 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Jan 14 12:46:53 2015 -0500

----------------------------------------------------------------------
 nifi-docs/src/main/asciidoc/images/new-flow.png | Bin 53989 -> 266913 bytes
 .../main/asciidoc/images/nifi-navigation.png    | Bin 85458 -> 339195 bytes
 .../asciidoc/images/nifi-toolbar-components.png | Bin 72263 -> 262314 bytes
 .../src/main/asciidoc/images/status-bar.png     | Bin 57171 -> 249077 bytes
 4 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a6f0948a/nifi-docs/src/main/asciidoc/images/new-flow.png
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/images/new-flow.png b/nifi-docs/src/main/asciidoc/images/new-flow.png
index 59f64d8..26a6b8f 100644
Binary files a/nifi-docs/src/main/asciidoc/images/new-flow.png and b/nifi-docs/src/main/asciidoc/images/new-flow.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a6f0948a/nifi-docs/src/main/asciidoc/images/nifi-navigation.png
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/images/nifi-navigation.png b/nifi-docs/src/main/asciidoc/images/nifi-navigation.png
index 1841cba..48c0f24 100644
Binary files a/nifi-docs/src/main/asciidoc/images/nifi-navigation.png and b/nifi-docs/src/main/asciidoc/images/nifi-navigation.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a6f0948a/nifi-docs/src/main/asciidoc/images/nifi-toolbar-components.png
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/images/nifi-toolbar-components.png b/nifi-docs/src/main/asciidoc/images/nifi-toolbar-components.png
index 3fd7170..9edaa4c 100644
Binary files a/nifi-docs/src/main/asciidoc/images/nifi-toolbar-components.png and b/nifi-docs/src/main/asciidoc/images/nifi-toolbar-components.png differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a6f0948a/nifi-docs/src/main/asciidoc/images/status-bar.png
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/images/status-bar.png b/nifi-docs/src/main/asciidoc/images/status-bar.png
index 7030eca..879be06 100644
Binary files a/nifi-docs/src/main/asciidoc/images/status-bar.png and b/nifi-docs/src/main/asciidoc/images/status-bar.png differ


[12/17] incubator-nifi git commit: NIFI-262, NIFI-263: Added 'restart' and 'dump' options to nifi.sh script

Posted by mc...@apache.org.
NIFI-262, NIFI-263: Added 'restart' and 'dump' options to nifi.sh script


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

Branch: refs/heads/NIFI-250
Commit: 7737fbd84d955520e2f6e23bbbfa8a5ebe43b3ed
Parents: c62aba1
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jan 14 12:24:09 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jan 14 12:24:09 2015 -0500

----------------------------------------------------------------------
 .../src/main/resources/bin/dump-nifi.bat        |  33 +++++
 .../resources/src/main/resources/bin/nifi.sh    |   4 +-
 .../java/org/apache/nifi/BootstrapListener.java | 122 ++++++++++++++++++-
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  89 ++++++++++++--
 4 files changed, 234 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7737fbd8/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/dump-nifi.bat
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/dump-nifi.bat b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/dump-nifi.bat
new file mode 100644
index 0000000..71e5a1a
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/dump-nifi.bat
@@ -0,0 +1,33 @@
+@echo off
+rem
+rem    Licensed to the Apache Software Foundation (ASF) under one or more
+rem    contributor license agreements.  See the NOTICE file distributed with
+rem    this work for additional information regarding copyright ownership.
+rem    The ASF licenses this file to You under the Apache License, Version 2.0
+rem    (the "License"); you may not use this file except in compliance with
+rem    the License.  You may obtain a copy of the License at
+rem
+rem       http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem    Unless required by applicable law or agreed to in writing, software
+rem    distributed under the License is distributed on an "AS IS" BASIS,
+rem    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem    See the License for the specific language governing permissions and
+rem    limitations under the License.
+rem
+
+rem Use JAVA_HOME if it's set; otherwise, just use java
+IF "%JAVA_HOME%"=="" (SET JAVA_EXE=java) ELSE (SET JAVA_EXE=%JAVA_HOME%\bin\java.exe)
+
+SET NIFI_ROOT=%~dp0..\
+CD /d "%NIFI_ROOT%"
+SET LIB_DIR=lib\bootstrap
+SET CONF_DIR=conf
+
+SET BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
+SET JAVA_ARGS=-Dorg.apache.nifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
+
+SET JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
+SET BOOTSTRAP_ACTION=dump
+
+cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7737fbd8/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
index 163f8e2..fb0d22e 100644
--- a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
+++ b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
@@ -172,10 +172,10 @@ case "$1" in
     install)
         install "$@"
         ;;
-    start|stop|run|status)
+    start|stop|run|restart|status|dump)
         main "$@"
         ;;
     *)
-        echo "Usage nifi {start|stop|run|status|install}"
+        echo "Usage nifi {start|stop|run|restart|status|dump|install}"
         ;;
 esac

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7737fbd8/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
index 3393952..590797c 100644
--- a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
+++ b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
@@ -17,16 +17,27 @@
 package org.apache.nifi;
 
 import java.io.BufferedReader;
+import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.lang.management.LockInfo;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MonitorInfo;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketTimeoutException;
 import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -162,6 +173,10 @@ public class BootstrapListener {
 										echoShutdown(socket.getOutputStream());
 										nifi.shutdownHook();
 										return;
+									case DUMP:
+									    logger.info("Received DUMP request from Bootstrap");
+									    writeDump(socket.getOutputStream());
+									    break;
 								}
 							} catch (final Throwable t) {
 								logger.error("Failed to process request from Bootstrap due to " + t.toString(), t);
@@ -182,6 +197,110 @@ public class BootstrapListener {
 	}
 	
 	
+	private static void writeDump(final OutputStream out) throws IOException {
+        final ThreadMXBean mbean = ManagementFactory.getThreadMXBean();
+        final BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+        
+        final ThreadInfo[] infos = mbean.dumpAllThreads(true, true);
+        final long[] deadlockedThreadIds = mbean.findDeadlockedThreads();
+        final long[] monitorDeadlockThreadIds = mbean.findMonitorDeadlockedThreads();
+        
+        final List<ThreadInfo> sortedInfos = new ArrayList<>(infos.length);
+        for ( final ThreadInfo info : infos ) {
+            sortedInfos.add(info);
+        }
+        Collections.sort(sortedInfos, new Comparator<ThreadInfo>() {
+            @Override
+            public int compare(ThreadInfo o1, ThreadInfo o2) {
+                return o1.getThreadName().toLowerCase().compareTo(o2.getThreadName().toLowerCase());
+            }
+        });
+        
+        final StringBuilder sb = new StringBuilder();
+        for ( final ThreadInfo info : sortedInfos ) {
+            sb.append("\n");
+            sb.append("\"").append(info.getThreadName()).append("\" Id=");
+            sb.append(info.getThreadId()).append(" ");
+            sb.append(info.getThreadState().toString()).append(" ");
+            
+            switch (info.getThreadState()) {
+                case BLOCKED:
+                case TIMED_WAITING:
+                case WAITING:
+                    sb.append(" on ");
+                    sb.append(info.getLockInfo());
+                    break;
+                default:
+                    break;
+            }
+            
+            if (info.isSuspended()) {
+                sb.append(" (suspended)");
+            }
+            if ( info.isInNative() ) {
+                sb.append(" (in native code)");
+            }
+            
+            if ( deadlockedThreadIds != null && deadlockedThreadIds.length > 0 ) {
+                for ( final long id : deadlockedThreadIds ) {
+                    if ( id == info.getThreadId() ) {
+                        sb.append(" ** DEADLOCKED THREAD **");
+                    }
+                }
+            }
+
+           if ( monitorDeadlockThreadIds != null && monitorDeadlockThreadIds.length > 0 ) {
+                for ( final long id : monitorDeadlockThreadIds ) {
+                    if ( id == info.getThreadId() ) {
+                        sb.append(" ** MONITOR-DEADLOCKED THREAD **");
+                    }
+                }
+            }
+
+            final StackTraceElement[] stackTraces = info.getStackTrace();
+            for ( final StackTraceElement element : stackTraces ) {
+                sb.append("\n\tat ").append(element);
+                
+                final MonitorInfo[] monitors = info.getLockedMonitors();
+                for ( final MonitorInfo monitor : monitors ) {
+                    if ( monitor.getLockedStackFrame().equals(element) ) {
+                        sb.append("\n\t- waiting on ").append(monitor);
+                    }
+                }
+            }
+            
+            final LockInfo[] lockInfos = info.getLockedSynchronizers();
+            if ( lockInfos.length > 0 ) {
+                sb.append("\n\t");
+                sb.append("Number of Locked Synchronizers: ").append(lockInfos.length);
+                for ( final LockInfo lockInfo : lockInfos ) {
+                    sb.append("\n\t- ").append(lockInfo.toString());
+                }
+            }
+            
+            sb.append("\n");
+        }
+        
+        if (deadlockedThreadIds != null && deadlockedThreadIds.length > 0) {
+            sb.append("\n\nDEADLOCK DETECTED!");
+            sb.append("\nThe following thread IDs are deadlocked:");
+            for ( final long id : deadlockedThreadIds ) {
+                sb.append("\n").append(id);
+            }
+        }
+
+       if (monitorDeadlockThreadIds != null && monitorDeadlockThreadIds.length > 0) {
+            sb.append("\n\nMONITOR DEADLOCK DETECTED!");
+            sb.append("\nThe following thread IDs are deadlocked:");
+            for ( final long id : monitorDeadlockThreadIds ) {
+                sb.append("\n").append(id);
+            }
+        }
+
+        writer.write(sb.toString());
+        writer.flush();
+    }
+	
 	private void echoPing(final OutputStream out) throws IOException {
 		out.write("PING\n".getBytes(StandardCharsets.UTF_8));
 		out.flush();
@@ -205,7 +324,7 @@ public class BootstrapListener {
 		
 		final String line = reader.readLine();
 		final String[] splits = line.split(" ");
-		if ( splits.length < 0 ) {
+		if ( splits.length < 1 ) {
 			throw new IOException("Received invalid request from Bootstrap: " + line);
 		}
 		
@@ -235,6 +354,7 @@ public class BootstrapListener {
 	private static class BootstrapRequest {
 		public static enum RequestType {
 			SHUTDOWN,
+			DUMP,
 			PING;
 		}
 		

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7737fbd8/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index e8f6439..f920860 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -34,6 +34,7 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.attribute.PosixFilePermission;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -75,6 +76,7 @@ public class RunNiFi {
 	
 	public static final String SHUTDOWN_CMD = "SHUTDOWN";
 	public static final String PING_CMD = "PING";
+	public static final String DUMP_CMD = "DUMP";
 	
 	private volatile boolean autoRestartNiFi = true;
 	private volatile int ccPort = -1;
@@ -105,41 +107,52 @@ public class RunNiFi {
 	private static void printUsage() {
 		System.out.println("Usage:");
 		System.out.println();
-		System.out.println("java org.apache.nifi.bootstrap.RunNiFi [<-verbose>] <command>");
+		System.out.println("java org.apache.nifi.bootstrap.RunNiFi [<-verbose>] <command> [options]");
 		System.out.println();
 		System.out.println("Valid commands include:");
 		System.out.println("");
 		System.out.println("Start : Start a new instance of Apache NiFi");
 		System.out.println("Stop : Stop a running instance of Apache NiFi");
+		System.out.println("Restart : Stop Apache NiFi, if it is running, and then start a new instance");
 		System.out.println("Status : Determine if there is a running instance of Apache NiFi");
+		System.out.println("Dump : Write a Thread Dump to the file specified by [options], or to the log if no file is given");
 		System.out.println("Run : Start a new instance of Apache NiFi and monitor the Process, restarting if the instance dies");
 		System.out.println();
 	}
 
+	private static String[] shift(final String[] orig) {
+	    return Arrays.copyOfRange(orig, 1, orig.length);
+	}
 	
-	public static void main(final String[] args) throws IOException, InterruptedException {
-		if ( args.length < 1 || args.length > 2 ) {
+	public static void main(String[] args) throws IOException, InterruptedException {
+		if ( args.length < 1 || args.length > 3 ) {
 			printUsage();
 			return;
 		}
 		
+		File dumpFile = null;
 		boolean verbose = false;
-		if ( args.length == 2 ) {
-		    if ( args[0].equals("-verbose") ) {
-		        verbose = true;
-		    } else {
-		        printUsage();
-		        return;
-		    }
+		if ( args[0].equals("-verbose") ) {
+		    verbose = true;
+		    args = shift(args);
 		}
 		
-		final String cmd = args.length == 1 ? args[0] : args[1];
+		final String cmd = args[0];
+	    if (cmd.equals("dump") ) {
+	        if ( args.length > 1 ) {
+	            dumpFile = new File(args[1]);
+	        } else {
+	            dumpFile = null;
+	        }
+	    }
 		
 		switch (cmd.toLowerCase()) {
 			case "start":
 			case "run":
 			case "stop":
 			case "status":
+			case "dump":
+			case "restart":
 				break;
 			default:
 				printUsage();
@@ -178,6 +191,13 @@ public class RunNiFi {
 			case "status":
 				runNiFi.status();
 				break;
+			case "restart":
+			    runNiFi.stop();
+			    runNiFi.start(false);
+			    break;
+			case "dump":
+			    runNiFi.dump(dumpFile);
+			    break;
 		}
 	}
 	
@@ -391,6 +411,53 @@ public class RunNiFi {
 	}
 	
 	
+	/**
+	 * Writes a NiFi thread dump to the given file; if file is null, logs at INFO level instead.
+	 * @param dumpFile
+	 * @return
+	 * @throws IOException
+	 */
+	public void dump(final File dumpFile) throws IOException {
+	    final Integer port = getCurrentPort();
+        if ( port == null ) {
+            System.out.println("Apache NiFi is not currently running");
+        }
+        
+        final Properties nifiProps = loadProperties();
+        final String secretKey = nifiProps.getProperty("secret.key");
+
+        final StringBuilder sb = new StringBuilder();
+	    try (final Socket socket = new Socket()) {
+            logger.fine("Connecting to NiFi instance");
+            socket.setSoTimeout(60000);
+            socket.connect(new InetSocketAddress("localhost", port));
+            logger.fine("Established connection to NiFi instance.");
+            socket.setSoTimeout(60000);
+            
+            logger.fine("Sending DUMP Command to port " + port);
+            final OutputStream out = socket.getOutputStream();
+            out.write((DUMP_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
+            
+            final InputStream in = socket.getInputStream();
+            final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+            String line;
+            while ((line = reader.readLine()) != null ) {
+                sb.append(line).append("\n");
+            }
+        }
+	    
+	    final String dump = sb.toString();
+	    if ( dumpFile == null ) {
+	        logger.info(dump);
+	    } else {
+	        try (final FileOutputStream fos = new FileOutputStream(dumpFile)) {
+	            fos.write(dump.getBytes(StandardCharsets.UTF_8));
+	        }
+	        logger.info("Successfully wrote thread dump to " + dumpFile.getAbsolutePath());
+	    }
+	}
+	
 	public void stop() throws IOException {
 		final Integer port = getCurrentPort();
 		if ( port == null ) {


[15/17] 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/7a958359
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/7a958359
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/7a958359

Branch: refs/heads/NIFI-250
Commit: 7a9583592729004e2783b60dbfc5fe461ddd17b0
Parents: d3aec88 a6f0948
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jan 14 13:28:39 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jan 14 13:28:39 2015 -0500

----------------------------------------------------------------------
 nifi-docs/src/main/asciidoc/images/new-flow.png | Bin 53989 -> 266913 bytes
 .../main/asciidoc/images/nifi-navigation.png    | Bin 85458 -> 339195 bytes
 .../asciidoc/images/nifi-toolbar-components.png | Bin 72263 -> 262314 bytes
 .../src/main/asciidoc/images/status-bar.png     | Bin 57171 -> 249077 bytes
 4 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------



[17/17] incubator-nifi git commit: Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250

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


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

Branch: refs/heads/NIFI-250
Commit: bb108a0960d6095791afd367bee91e62d8d605da
Parents: da18ce0 03d422e
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Jan 14 13:34:45 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Jan 14 13:34:45 2015 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------