You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2017/11/07 15:50:05 UTC

[1/2] nifi-registry git commit: NIFIREG-47 Improvements to nifi-registry-cleitn - Using separate Jersey version for client only - Added support for proxied entities - Fixed deserializing of BucketItems - Cleaned up methods with unnecessary parameters

Repository: nifi-registry
Updated Branches:
  refs/heads/master f07fd2f5c -> a43e81fed


NIFIREG-47 Improvements to nifi-registry-cleitn
- Using separate Jersey version for client only
- Added support for proxied entities
- Fixed deserializing of BucketItems
- Cleaned up methods with unnecessary parameters


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

Branch: refs/heads/master
Commit: 071d08aacccad125f4f17aacd97dbcbca9311779
Parents: f07fd2f
Author: Bryan Bende <bb...@apache.org>
Authored: Thu Nov 2 15:05:30 2017 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Nov 7 10:50:00 2017 -0500

----------------------------------------------------------------------
 nifi-registry-client/pom.xml                    |  46 +++++++-
 .../apache/nifi/registry/client/FlowClient.java |   3 +-
 .../registry/client/NiFiRegistryClient.java     |  20 ++++
 .../client/impl/AbstractJerseyClient.java       |  29 +++++
 .../client/impl/BucketItemDeserializer.java     |  76 ++++++++++++
 .../client/impl/JerseyBucketClient.java         |  47 +++++---
 .../registry/client/impl/JerseyFlowClient.java  |  62 +++++-----
 .../client/impl/JerseyFlowSnapshotClient.java   |  59 ++++++----
 .../registry/client/impl/JerseyItemsClient.java |  19 ++-
 .../client/impl/JerseyNiFiRegistryClient.java   | 117 ++++++++++++++++---
 .../impl/TestJerseyNiFiRegistryClient.java      |  41 ++++++-
 .../authorization/AuthorizableLookup.java       |   8 ++
 .../StandardAuthorizableLookup.java             |  19 +++
 .../TestFlowSnapshotSerializer.java             |  29 +++--
 .../file/FileAccessPolicyProvider.java          |   1 +
 nifi-registry-web-api/pom.xml                   |   5 -
 .../registry/web/api/AccessPolicyResource.java  |   2 +-
 17 files changed, 470 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-client/pom.xml b/nifi-registry-client/pom.xml
index 6627285..6605995 100644
--- a/nifi-registry-client/pom.xml
+++ b/nifi-registry-client/pom.xml
@@ -21,10 +21,44 @@
     <artifactId>nifi-registry-client</artifactId>
     <packaging>jar</packaging>
 
+    <!-- Use the newest version of Jersey for the client here, since the primary client is NiFi which also uses Jersey 2.26,
+         when spring-boot 2.0.0 comes out spring-boot-jersey will use 2.26 and nifi-registry-client and backend can both use 2.2.6
+     -->
     <properties>
         <jersey.version>2.26</jersey.version>
+        <jax.rs.api.version>2.1</jax.rs.api.version>
     </properties>
-    
+
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>javax.ws.rs</groupId>
+                <artifactId>javax.ws.rs-api</artifactId>
+                <version>${jax.rs.api.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.glassfish.jersey.core</groupId>
+                <artifactId>jersey-client</artifactId>
+                <version>${jersey.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.glassfish.jersey.media</groupId>
+                <artifactId>jersey-media-json-jackson</artifactId>
+                <version>${jersey.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.glassfish.jersey.inject</groupId>
+                <artifactId>jersey-hk2</artifactId>
+                <version>${jersey.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.glassfish.jersey.core</groupId>
+                <artifactId>jersey-common</artifactId>
+                <version>${jersey.version}</version>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi.registry</groupId>
@@ -39,12 +73,18 @@
         <dependency>
             <groupId>org.glassfish.jersey.core</groupId>
             <artifactId>jersey-client</artifactId>
-            <version>${jersey.version}</version>
         </dependency>
         <dependency>
             <groupId>org.glassfish.jersey.media</groupId>
             <artifactId>jersey-media-json-jackson</artifactId>
-            <version>${jersey.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.inject</groupId>
+            <artifactId>jersey-hk2</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-common</artifactId>
         </dependency>
         <dependency>
             <groupId>org.slf4j</groupId>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/FlowClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/FlowClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/FlowClient.java
index 206c4f1..f9fdf4e 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/FlowClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/FlowClient.java
@@ -31,13 +31,12 @@ public interface FlowClient {
     /**
      * Create the given flow in the given bucket.
      *
-     * @param bucketId a bucket id
      * @param flow the flow to create
      * @return the created flow with the identifier populated
      * @throws NiFiRegistryException if an error is encountered other than IOException
      * @throws IOException if an I/O error is encountered
      */
-    VersionedFlow create(String bucketId, VersionedFlow flow) throws NiFiRegistryException, IOException;
+    VersionedFlow create(VersionedFlow flow) throws NiFiRegistryException, IOException;
 
     /**
      * Gets the flow with the given id in the given bucket.

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
index f53acf4..9ee1573 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
@@ -29,21 +29,41 @@ public interface NiFiRegistryClient extends Closeable {
     BucketClient getBucketClient();
 
     /**
+     * @return the client for interacting with buckets on behalf of the given proxied entities
+     */
+    BucketClient getBucketClient(String ... proxiedEntity);
+
+    /**
      * @return the client for interacting with flows
      */
     FlowClient getFlowClient();
 
     /**
+     * @return the client for interacting with flows on behalf of the given proxied entities
+     */
+    FlowClient getFlowClient(String ... proxiedEntity);
+
+    /**
      * @return the client for interacting with flows/snapshots
      */
     FlowSnapshotClient getFlowSnapshotClient();
 
     /**
+     * @return the client for interacting with flows/snapshots on behalf of the given proxied entities
+     */
+    FlowSnapshotClient getFlowSnapshotClient(String ... proxiedEntity);
+
+    /**
      * @return the client for interacting with bucket items
      */
     ItemsClient getItemsClient();
 
     /**
+     * @return the client for interacting with bucket items on behalf of the given proxied entities
+     */
+    ItemsClient getItemsClient(String ... proxiedEntity);
+
+    /**
      * The builder interface that implementations should provide for obtaining the client.
      */
     interface Builder {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractJerseyClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractJerseyClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractJerseyClient.java
index ca28abf..b31cdfa 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractJerseyClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractJerseyClient.java
@@ -18,13 +18,42 @@ package org.apache.nifi.registry.client.impl;
 
 import org.apache.nifi.registry.client.NiFiRegistryException;
 
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
 import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * Base class for the client operations to share exception handling.
+ *
+ * Sub-classes should always execute a request from getRequestBuilder(target) to ensure proper headers are sent.
  */
 public class AbstractJerseyClient {
 
+    private final Map<String,String> headers;
+
+    public AbstractJerseyClient(final Map<String, String> headers) {
+        this.headers = headers == null ? Collections.emptyMap() : Collections.unmodifiableMap(new HashMap<>(headers));
+    }
+
+    protected Map<String,String> getHeaders() {
+        return headers;
+    }
+
+    /**
+     * Creates a new Invocation.Builder for the given WebTarget with the headers added to the builder.
+     *
+     * @param webTarget the target for the request
+     * @return the builder for the target with the headers added
+     */
+    protected Invocation.Builder getRequestBuilder(final WebTarget webTarget) {
+        final Invocation.Builder requestBuilder = webTarget.request();
+        headers.entrySet().stream().forEach(e -> requestBuilder.header(e.getKey(), e.getValue()));
+        return requestBuilder;
+    }
+
     /**
      * Executes the given action and returns the result.
      *

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java
new file mode 100644
index 0000000..4ea5005
--- /dev/null
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/BucketItemDeserializer.java
@@ -0,0 +1,76 @@
+/*
+ * 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.registry.client.impl;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.bucket.BucketItem;
+import org.apache.nifi.registry.bucket.BucketItemType;
+import org.apache.nifi.registry.flow.VersionedFlow;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class BucketItemDeserializer extends StdDeserializer<BucketItem[]> {
+
+    public BucketItemDeserializer() {
+        super(BucketItem[].class);
+    }
+
+    @Override
+    public BucketItem[] deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+        final JsonNode arrayNode = jsonParser.getCodec().readTree(jsonParser);
+
+        final List<BucketItem> bucketItems = new ArrayList<>();
+
+        final Iterator<JsonNode> nodeIter = arrayNode.elements();
+        while (nodeIter.hasNext()) {
+            final JsonNode node = nodeIter.next();
+
+            final String type = node.get("type").asText();
+            if (StringUtils.isBlank(type)) {
+                throw new IllegalStateException("BucketItem type cannot be null or blank");
+            }
+
+            final BucketItemType bucketItemType;
+            try {
+                bucketItemType = BucketItemType.valueOf(type);
+            } catch (Exception e) {
+                throw new IllegalStateException("Unknown type for BucketItem: " + type, e);
+            }
+
+
+            switch (bucketItemType) {
+                case FLOW:
+                    final VersionedFlow versionedFlow = jsonParser.getCodec().treeToValue(node, VersionedFlow.class);
+                    bucketItems.add(versionedFlow);
+                    break;
+                default:
+                    throw new IllegalStateException("Unknown type for BucketItem");
+            }
+        }
+
+        return bucketItems.toArray(new BucketItem[bucketItems.size()]);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
index a2798ce..b1734a6 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
@@ -28,7 +28,9 @@ import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.MediaType;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Jersey implementation of BucketClient.
@@ -37,7 +39,13 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
 
     private final WebTarget bucketsTarget;
 
+
     public JerseyBucketClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyBucketClient(final WebTarget baseTarget, final Map<String,String> headers) {
+        super(headers);
         this.bucketsTarget = baseTarget.path("/buckets");
     }
 
@@ -48,7 +56,7 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
         }
 
         return executeAction("Error creating bucket", () -> {
-            return bucketsTarget.request()
+            return getRequestBuilder(bucketsTarget)
                     .post(
                             Entity.entity(bucket, MediaType.APPLICATION_JSON),
                             Bucket.class
@@ -64,11 +72,11 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
         }
 
         return executeAction("Error retrieving bucket", () -> {
-            return bucketsTarget
+            final WebTarget target = bucketsTarget
                     .path("/{bucketId}")
-                    .resolveTemplate("bucketId", bucketId)
-                    .request()
-                    .get(Bucket.class);
+                    .resolveTemplate("bucketId", bucketId);
+
+            return getRequestBuilder(target).get(Bucket.class);
         });
 
     }
@@ -84,10 +92,11 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
         }
 
         return executeAction("Error updating bucket", () -> {
-            return bucketsTarget
+            final WebTarget target = bucketsTarget
                     .path("/{bucketId}")
-                    .resolveTemplate("bucketId", bucket.getIdentifier())
-                    .request()
+                    .resolveTemplate("bucketId", bucket.getIdentifier());
+
+            return getRequestBuilder(target)
                     .put(
                             Entity.entity(bucket, MediaType.APPLICATION_JSON),
                             Bucket.class
@@ -103,28 +112,29 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
         }
 
         return executeAction("Error deleting bucket", () -> {
-            return bucketsTarget
+            final WebTarget target = bucketsTarget
                     .path("/{bucketId}")
-                    .resolveTemplate("bucketId", bucketId)
-                    .request()
-                    .delete(Bucket.class);
+                    .resolveTemplate("bucketId", bucketId);
+
+            return getRequestBuilder(target).delete(Bucket.class);
         });
     }
 
     @Override
     public Fields getFields() throws NiFiRegistryException, IOException {
         return executeAction("Error retrieving bucket field info", () -> {
-            return bucketsTarget
-                    .path("/fields")
-                    .request()
-                    .get(Fields.class);
+            final WebTarget target = bucketsTarget
+                    .path("/fields");
+
+            return getRequestBuilder(target).get(Fields.class);
         });
     }
 
     @Override
     public List<Bucket> getAll() throws NiFiRegistryException, IOException {
         return executeAction("Error retrieving all buckets", () -> {
-            return Arrays.asList(bucketsTarget.request().get(Bucket[].class));
+            final Bucket[] buckets = getRequestBuilder(bucketsTarget).get(Bucket[].class);
+            return buckets == null ? Collections.emptyList() : Arrays.asList(buckets);
         });
     }
 
@@ -140,7 +150,8 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
                 target = target.queryParam("sort", sortParam.toString());
             }
 
-            return Arrays.asList(target.request().get(Bucket[].class));
+            return getRequestBuilder(target).get(List.class);
         });
     }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowClient.java
index 977a98f..696394e 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowClient.java
@@ -27,8 +27,10 @@ import javax.ws.rs.client.Entity;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.MediaType;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Jersey implementation of FlowClient.
@@ -39,12 +41,18 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
     private final WebTarget bucketFlowsTarget;
 
     public JerseyFlowClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyFlowClient(final WebTarget baseTarget, final Map<String,String> headers) {
+        super(headers);
         this.flowsTarget = baseTarget.path("/flows");
         this.bucketFlowsTarget = baseTarget.path("/buckets/{bucketId}/flows");
     }
 
     @Override
-    public VersionedFlow create(final String bucketId, final VersionedFlow flow) throws NiFiRegistryException, IOException {
+    public VersionedFlow create(final VersionedFlow flow) throws NiFiRegistryException, IOException {
+        final String bucketId = flow.getBucketIdentifier();
         if (StringUtils.isBlank(bucketId)) {
             throw new IllegalArgumentException("Bucket Identifier cannot be blank");
         }
@@ -54,9 +62,10 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
         }
 
         return executeAction("Error creating flow", () -> {
-            return bucketFlowsTarget
-                    .resolveTemplate("bucketId", bucketId)
-                    .request()
+            final WebTarget target = bucketFlowsTarget
+                    .resolveTemplate("bucketId", bucketId);
+
+            return getRequestBuilder(target)
                     .post(
                             Entity.entity(flow, MediaType.APPLICATION_JSON),
                             VersionedFlow.class
@@ -75,12 +84,12 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
         }
 
         return executeAction("Error retrieving flow", () -> {
-            return bucketFlowsTarget
+            final WebTarget target = bucketFlowsTarget
                     .path("/{flowId}")
                     .resolveTemplate("bucketId", bucketId)
-                    .resolveTemplate("flowId", flowId)
-                    .request()
-                    .get(VersionedFlow.class);
+                    .resolveTemplate("flowId", flowId);
+
+            return  getRequestBuilder(target).get(VersionedFlow.class);
         });
     }
 
@@ -95,13 +104,13 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
         }
 
         return executeAction("Error retrieving flow", () -> {
-            return bucketFlowsTarget
+            final WebTarget target = bucketFlowsTarget
                     .path("/{flowId}")
                     .resolveTemplate("bucketId", bucketId)
                     .resolveTemplate("flowId", flowId)
-                    .queryParam("verbose", "true")
-                    .request()
-                    .get(VersionedFlow.class);
+                    .queryParam("verbose", "true");
+
+            return getRequestBuilder(target).get(VersionedFlow.class);
         });
     }
 
@@ -120,11 +129,12 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
         }
 
         return executeAction("Error updating flow", () -> {
-            return bucketFlowsTarget
+            final WebTarget target = bucketFlowsTarget
                     .path("/{flowId}")
                     .resolveTemplate("bucketId", bucketId)
-                    .resolveTemplate("flowId", flow.getIdentifier())
-                    .request()
+                    .resolveTemplate("flowId", flow.getIdentifier());
+
+            return  getRequestBuilder(target)
                     .put(
                             Entity.entity(flow, MediaType.APPLICATION_JSON),
                             VersionedFlow.class
@@ -143,22 +153,20 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
         }
 
         return executeAction("Error deleting flow", () -> {
-            return bucketFlowsTarget
+            final WebTarget target = bucketFlowsTarget
                     .path("/{flowId}")
                     .resolveTemplate("bucketId", bucketId)
-                    .resolveTemplate("flowId", flowId)
-                    .request()
-                    .delete(VersionedFlow.class);
+                    .resolveTemplate("flowId", flowId);
+
+            return getRequestBuilder(target).delete(VersionedFlow.class);
         });
     }
 
     @Override
     public Fields getFields() throws NiFiRegistryException, IOException {
         return executeAction("Error retrieving fields info for flows", () -> {
-            return flowsTarget
-                    .path("/fields")
-                    .request()
-                    .get(Fields.class);
+            final WebTarget target = flowsTarget.path("/fields");
+            return getRequestBuilder(target).get(Fields.class);
         });
     }
 
@@ -183,10 +191,10 @@ public class JerseyFlowClient extends AbstractJerseyClient  implements FlowClien
                 target = target.queryParam("sort", sortParam.toString());
             }
 
-            return target
-                    .resolveTemplate("bucketId", bucketId)
-                    .request()
-                    .get(List.class);
+            target = target.resolveTemplate("bucketId", bucketId);
+
+            final VersionedFlow[] versionedFlows = getRequestBuilder(target).get(VersionedFlow[].class);
+            return  versionedFlows == null ? Collections.emptyList() : Arrays.asList(versionedFlows);
         });
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowSnapshotClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowSnapshotClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowSnapshotClient.java
index c3685d3..bf9ad2d 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowSnapshotClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyFlowSnapshotClient.java
@@ -26,8 +26,10 @@ import javax.ws.rs.client.Entity;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.MediaType;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
-import java.util.Objects;
+import java.util.Map;
 
 /**
  * Jersey implementation of FlowSnapshotClient.
@@ -37,34 +39,38 @@ public class JerseyFlowSnapshotClient extends AbstractJerseyClient implements Fl
     final WebTarget flowSnapshotTarget;
 
     public JerseyFlowSnapshotClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyFlowSnapshotClient(final WebTarget baseTarget, final Map<String,String> headers) {
+        super(headers);
         this.flowSnapshotTarget = baseTarget.path("/buckets/{bucketId}/flows/{flowId}/versions");
     }
 
-    @Override
-    public VersionedFlowSnapshot create(final VersionedFlowSnapshot snapshot) throws NiFiRegistryException, IOException {
-        Objects.requireNonNull(snapshot);
 
-        final VersionedFlowSnapshotMetadata metadata = snapshot.getSnapshotMetadata();
-        if (metadata == null) {
-            throw new IllegalArgumentException("Snapshot Metadata must be supplied.");
+    @Override
+    public VersionedFlowSnapshot create(final VersionedFlowSnapshot snapshot)
+            throws NiFiRegistryException, IOException {
+        if (snapshot.getSnapshotMetadata() == null) {
+            throw new IllegalArgumentException("Snapshot Metadata cannot be null");
         }
 
-        final String bucketId = metadata.getBucketIdentifier();
-        final String flowId = metadata.getFlowIdentifier();
-
+        final String bucketId = snapshot.getSnapshotMetadata().getBucketIdentifier();
         if (StringUtils.isBlank(bucketId)) {
             throw new IllegalArgumentException("Bucket Identifier cannot be blank");
         }
 
+        final String flowId = snapshot.getSnapshotMetadata().getFlowIdentifier();
         if (StringUtils.isBlank(flowId)) {
             throw new IllegalArgumentException("Flow Identifier cannot be blank");
         }
 
         return executeAction("Error creating snapshot", () -> {
-            return flowSnapshotTarget
+            final WebTarget target = flowSnapshotTarget
                     .resolveTemplate("bucketId", bucketId)
-                    .resolveTemplate("flowId", flowId)
-                    .request()
+                    .resolveTemplate("flowId", flowId);
+
+            return  getRequestBuilder(target)
                     .post(
                             Entity.entity(snapshot, MediaType.APPLICATION_JSON),
                             VersionedFlowSnapshot.class
@@ -88,13 +94,13 @@ public class JerseyFlowSnapshotClient extends AbstractJerseyClient implements Fl
         }
 
         return executeAction("Error retrieving flow snapshot", () -> {
-            return flowSnapshotTarget
+            final WebTarget target = flowSnapshotTarget
                     .path("/{version}")
                     .resolveTemplate("bucketId", bucketId)
                     .resolveTemplate("flowId", flowId)
-                    .resolveTemplate("version", version)
-                    .request()
-                    .get(VersionedFlowSnapshot.class);
+                    .resolveTemplate("version", version);
+
+            return getRequestBuilder(target).get(VersionedFlowSnapshot.class);
         });
     }
 
@@ -110,12 +116,12 @@ public class JerseyFlowSnapshotClient extends AbstractJerseyClient implements Fl
         }
 
         return executeAction("Error retrieving latest snapshot", () -> {
-            return flowSnapshotTarget
+            final WebTarget target = flowSnapshotTarget
                     .path("/latest")
                     .resolveTemplate("bucketId", bucketId)
-                    .resolveTemplate("flowId", flowId)
-                    .request()
-                    .get(VersionedFlowSnapshot.class);
+                    .resolveTemplate("flowId", flowId);
+
+            return getRequestBuilder(target).get(VersionedFlowSnapshot.class);
         });
     }
 
@@ -132,11 +138,14 @@ public class JerseyFlowSnapshotClient extends AbstractJerseyClient implements Fl
         }
 
         return executeAction("Error retrieving snapshot metadata", () -> {
-            return flowSnapshotTarget
+            final WebTarget target = flowSnapshotTarget
                     .resolveTemplate("bucketId", bucketId)
-                    .resolveTemplate("flowId", flowId)
-                    .request()
-                    .get(List.class);
+                    .resolveTemplate("flowId", flowId);
+
+            final VersionedFlowSnapshotMetadata[] snapshots = getRequestBuilder(target)
+                    .get(VersionedFlowSnapshotMetadata[].class);
+
+            return snapshots == null ? Collections.emptyList() : Arrays.asList(snapshots);
         });
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyItemsClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyItemsClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyItemsClient.java
index 2715668..05fb613 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyItemsClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyItemsClient.java
@@ -25,8 +25,10 @@ import org.apache.nifi.registry.params.SortParameter;
 
 import javax.ws.rs.client.WebTarget;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Jersey implementation of ItemsClient.
@@ -36,6 +38,11 @@ public class JerseyItemsClient extends AbstractJerseyClient implements ItemsClie
     private final WebTarget itemsTarget;
 
     public JerseyItemsClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyItemsClient(final WebTarget baseTarget, final Map<String,String> headers) {
+        super(headers);
         this.itemsTarget = baseTarget.path("/items");
     }
 
@@ -56,7 +63,8 @@ public class JerseyItemsClient extends AbstractJerseyClient implements ItemsClie
                 target = target.queryParam("sort", sortParam.toString());
             }
 
-            return target.request().get(List.class);
+            final BucketItem[] bucketItems = getRequestBuilder(target).get(BucketItem[].class);
+            return bucketItems == null ? Collections.emptyList() : Arrays.asList(bucketItems);
         });
     }
 
@@ -85,17 +93,16 @@ public class JerseyItemsClient extends AbstractJerseyClient implements ItemsClie
                 target = target.queryParam("sort", sortParam.toString());
             }
 
-            return target.request().get(List.class);
+            final BucketItem[] bucketItems = getRequestBuilder(target).get(BucketItem[].class);
+            return bucketItems == null ? Collections.emptyList() : Arrays.asList(bucketItems);
         });
     }
 
     @Override
     public Fields getFields() throws NiFiRegistryException, IOException {
         return executeAction("", () -> {
-            return itemsTarget
-                    .path("/fields")
-                    .request()
-                    .get(Fields.class);
+            final WebTarget target = itemsTarget.path("/fields");
+            return getRequestBuilder(target).get(Fields.class);
 
         });
     }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
index 41efb75..1061455 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
@@ -16,16 +16,13 @@
  */
 package org.apache.nifi.registry.client.impl;
 
-import java.io.IOException;
-import java.net.URI;
-
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.SSLContext;
-import javax.ws.rs.client.Client;
-import javax.ws.rs.client.ClientBuilder;
-import javax.ws.rs.client.WebTarget;
-
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.client.BucketClient;
 import org.apache.nifi.registry.client.FlowClient;
 import org.apache.nifi.registry.client.FlowSnapshotClient;
@@ -36,21 +33,38 @@ import org.glassfish.jersey.client.ClientConfig;
 import org.glassfish.jersey.client.ClientProperties;
 import org.glassfish.jersey.jackson.internal.jackson.jaxrs.json.JacksonJaxbJsonProvider;
 
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLContext;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
 
 /**
  * A NiFiRegistryClient that uses Jersey Client.
  */
 public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
 
+    static final String PROXY_ENTITIES_CHAIN = "X-ProxiedEntitiesChain";
+
+    static final String GT = ">";
+    static final String ESCAPED_GT = "\\\\>";
+    static final String LT = "<";
+    static final String ESCAPED_LT = "\\\\<";
+
     static final String NIFI_REGISTRY_CONTEXT = "nifi-registry-api";
     static final int DEFAULT_CONNECT_TIMEOUT = 10000;
     static final int DEFAULT_READ_TIMEOUT = 10000;
 
     private final Client client;
+    private final WebTarget baseTarget;
+
     private final BucketClient bucketClient;
     private final FlowClient flowClient;
     private final FlowSnapshotClient flowSnapshotClient;
@@ -102,7 +116,7 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
         clientBuilder.withConfig(clientConfig);
         this.client = clientBuilder.build();
 
-        final WebTarget baseTarget = client.target(baseUrl);
+        this.baseTarget = client.target(baseUrl);
         this.bucketClient = new JerseyBucketClient(baseTarget);
         this.flowClient = new JerseyFlowClient(baseTarget);
         this.flowSnapshotClient = new JerseyFlowSnapshotClient(baseTarget);
@@ -130,6 +144,77 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
     }
 
     @Override
+    public BucketClient getBucketClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyBucketClient(baseTarget, headers);
+    }
+
+    @Override
+    public FlowClient getFlowClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyFlowClient(baseTarget, headers);
+    }
+
+    @Override
+    public FlowSnapshotClient getFlowSnapshotClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyFlowSnapshotClient(baseTarget, headers);
+    }
+
+    @Override
+    public ItemsClient getItemsClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyItemsClient(baseTarget, headers);
+    }
+
+    private Map<String,String> getHeaders(String[] proxiedEntities) {
+        final String proxiedEntitiesValue = getProxiedEntitesValue(proxiedEntities);
+
+        final Map<String,String> headers = new HashMap<>();
+        if (proxiedEntitiesValue != null) {
+            headers.put(PROXY_ENTITIES_CHAIN, proxiedEntitiesValue);
+        }
+        return headers;
+    }
+
+    private String getProxiedEntitesValue(String[] proxiedEntities) {
+        if (proxiedEntities == null) {
+            return null;
+        }
+
+        final List<String> proxiedEntityChain = Arrays.asList(proxiedEntities).stream().map(dn -> formatProxyDn(dn)).collect(Collectors.toList());
+        return StringUtils.join(proxiedEntityChain, "");
+    }
+
+    /**
+     * Formats the specified DN to be set as a HTTP header using well known conventions.
+     *
+     * @param dn raw dn
+     * @return the dn formatted as an HTTP header
+     */
+    private static String formatProxyDn(String dn) {
+        return LT + sanitizeDn(dn) + GT;
+    }
+
+    /**
+     * If a user provides a DN with the sequence '><', they could escape the tokenization process and impersonate another user.
+     * <p>
+     * Example:
+     * <p>
+     * Provided DN: {@code jdoe><alopresto} -> {@code <jdoe><alopresto><proxy...>} would allow the user to impersonate jdoe
+     *
+     * @param rawDn the unsanitized DN
+     * @return the sanitized DN
+     */
+    private static String sanitizeDn(String rawDn) {
+        if (StringUtils.isEmpty(rawDn)) {
+            return rawDn;
+        } else {
+            return rawDn.replaceAll(GT, ESCAPED_GT).replaceAll(LT, ESCAPED_LT);
+        }
+    }
+
+    @Override
     public void close() throws IOException {
         if (this.client != null) {
             try {
@@ -174,6 +259,10 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
         // Ignore unknown properties so that deployed client remain compatible with future versions of NiFi Registry that add new fields
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
 
+        SimpleModule module = new SimpleModule();
+        module.addDeserializer(BucketItem[].class, new BucketItemDeserializer());
+        mapper.registerModule(module);
+
         jacksonJaxbJsonProvider.setMapper(mapper);
         return jacksonJaxbJsonProvider;
     }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java b/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
index 1fc0049..e813193 100644
--- a/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
+++ b/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
@@ -30,6 +30,7 @@ import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
+import org.apache.nifi.registry.flow.VersionedProcessor;
 import org.apache.nifi.registry.params.SortOrder;
 import org.apache.nifi.registry.params.SortParameter;
 import org.junit.Assert;
@@ -39,7 +40,9 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 public class TestJerseyNiFiRegistryClient {
 
@@ -89,6 +92,7 @@ public class TestJerseyNiFiRegistryClient {
             final List<Bucket> allBuckets = bucketClient.getAll();
             System.out.println("Retrieved buckets, size = " + allBuckets.size());
             Assert.assertEquals(numBuckets, allBuckets.size());
+            allBuckets.stream().forEach(b -> System.out.println("Retrieve bucket " + b.getIdentifier()));
 
             // get all buckets with sorting
             final SortParameter sortParam = new SortParameter("created", SortOrder.ASC);
@@ -148,6 +152,7 @@ public class TestJerseyNiFiRegistryClient {
             final List<VersionedFlow> flowsInBucket = flowClient.getByBucket(flowsBucket.getIdentifier());
             Assert.assertNotNull(flowsInBucket);
             Assert.assertEquals(2, flowsInBucket.size());
+            flowsInBucket.stream().forEach(f -> System.out.println("Flow in bucket, flow id " + f.getIdentifier()));
 
             // get flows in bucket with sorting
             final SortParameter flowsSortParam = new SortParameter("created", SortOrder.ASC);
@@ -187,6 +192,7 @@ public class TestJerseyNiFiRegistryClient {
             final List<VersionedFlowSnapshotMetadata> retrievedMetadata = snapshotClient.getSnapshotMetadata(snapshotFlow.getBucketIdentifier(), snapshotFlow.getIdentifier());
             Assert.assertNotNull(retrievedMetadata);
             Assert.assertEquals(2, retrievedMetadata.size());
+            retrievedMetadata.stream().forEach(s -> System.out.println("Retrieved snapshot metadata " + s.getVersion()));
 
             // ---------------------- TEST ITEMS --------------------------//
 
@@ -200,6 +206,7 @@ public class TestJerseyNiFiRegistryClient {
             // get all items
             final List<BucketItem> allItems = itemsClient.getAll();
             Assert.assertEquals(2, allItems.size());
+            allItems.stream().forEach(i -> System.out.println("All items, item " + i.getIdentifier()));
 
             // get all items with sorting
             final SortParameter itemsSortParam = new SortParameter("created", SortOrder.ASC);
@@ -209,6 +216,7 @@ public class TestJerseyNiFiRegistryClient {
             // get items for bucket
             final List<BucketItem> bucketItems = itemsClient.getByBucket(flowsBucket.getIdentifier());
             Assert.assertEquals(2, bucketItems.size());
+            bucketItems.stream().forEach(i -> System.out.println("Items in bucket, item " + i.getIdentifier()));
 
             // get items for bucket with sorting
             final List<BucketItem> bucketItemsSorted = itemsClient.getByBucket(flowsBucket.getIdentifier(), Arrays.asList(itemsSortParam));
@@ -256,7 +264,8 @@ public class TestJerseyNiFiRegistryClient {
         final VersionedFlow versionedFlow = new VersionedFlow();
         versionedFlow.setName(bucket.getName() + " Flow #" + num);
         versionedFlow.setDescription("This is " + bucket.getName() + " flow #" + num);
-        return client.create(bucket.getIdentifier(), versionedFlow);
+        versionedFlow.setBucketIdentifier(bucket.getIdentifier());
+        return client.create(versionedFlow);
     }
 
     private static VersionedFlowSnapshot createSnapshot(FlowSnapshotClient client, VersionedFlow flow, int num) throws IOException, NiFiRegistryException {
@@ -267,13 +276,35 @@ public class TestJerseyNiFiRegistryClient {
         snapshotMetadata.setVersion(num);
         snapshotMetadata.setComments("This is snapshot #" + num);
 
-        final VersionedProcessGroup snapshotContents = new VersionedProcessGroup();
-        snapshotContents.setIdentifier("pg1");
-        snapshotContents.setName("Process Group 1");
+        final VersionedProcessGroup rootProcessGroup = new VersionedProcessGroup();
+        rootProcessGroup.setIdentifier("root-pg");
+        rootProcessGroup.setName("Root Process Group");
+
+        final VersionedProcessGroup subProcessGroup = new VersionedProcessGroup();
+        subProcessGroup.setIdentifier("sub-pg");
+        subProcessGroup.setName("Sub Process Group");
+        rootProcessGroup.getProcessGroups().add(subProcessGroup);
+
+        final Map<String,String> processorProperties = new HashMap<>();
+        processorProperties.put("Prop 1", "Val 1");
+        processorProperties.put("Prop 2", "Val 2");
+
+        final VersionedProcessor processor1 = new VersionedProcessor();
+        processor1.setIdentifier("p1");
+        processor1.setName("Processor 1");
+        processor1.setProperties(processorProperties);
+
+        final VersionedProcessor processor2 = new VersionedProcessor();
+        processor2.setIdentifier("p2");
+        processor2.setName("Processor 2");
+        processor2.setProperties(processorProperties);
+
+        subProcessGroup.getProcessors().add(processor1);
+        subProcessGroup.getProcessors().add(processor2);
 
         final VersionedFlowSnapshot snapshot = new VersionedFlowSnapshot();
         snapshot.setSnapshotMetadata(snapshotMetadata);
-        snapshot.setFlowContents(snapshotContents);
+        snapshot.setFlowContents(rootProcessGroup);
 
         return client.create(snapshot);
     }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
index 94a74bc..e5016d3 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
@@ -19,6 +19,7 @@ package org.apache.nifi.registry.authorization;
 import org.apache.nifi.registry.authorization.resource.Authorizable;
 
 public interface AuthorizableLookup {
+
     /**
      * Get the authorizable for retrieving resources.
      *
@@ -27,6 +28,13 @@ public interface AuthorizableLookup {
     Authorizable getResourcesAuthorizable();
 
     /**
+     * Get the authorizable for /proxy.
+     *
+     * @return authorizable
+     */
+    Authorizable getProxyAuthorizable();
+
+    /**
      * Get the authorizable for all tenants.
      *
      * Get the {@link Authorizable} that represents the resource of users and user groups.

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
index 29339c7..78cd630 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
@@ -74,12 +74,29 @@ public class StandardAuthorizableLookup implements AuthorizableLookup {
         }
     };
 
+    private static final Authorizable PROXY_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getProxyResource();
+        }
+    };
+
     @Override
     public Authorizable getResourcesAuthorizable() {
         return RESOURCES_AUTHORIZABLE;
     }
 
     @Override
+    public Authorizable getProxyAuthorizable() {
+        return PROXY_AUTHORIZABLE;
+    }
+
+    @Override
     public Authorizable getTenantsAuthorizable() {
         return TENANTS_AUTHORIZABLE;
     }
@@ -206,6 +223,8 @@ public class StandardAuthorizableLookup implements AuthorizableLookup {
             case Tenant:
                 authorizable = getTenantsAuthorizable();
                 break;
+            case Proxy:
+                authorizable = getProxyAuthorizable();
         }
 
         if (authorizable == null) {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-framework/src/test/java/org/apache/nifi/registry/serialization/TestFlowSnapshotSerializer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/serialization/TestFlowSnapshotSerializer.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/serialization/TestFlowSnapshotSerializer.java
index 14c78b2..014f932 100644
--- a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/serialization/TestFlowSnapshotSerializer.java
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/serialization/TestFlowSnapshotSerializer.java
@@ -19,6 +19,7 @@ package org.apache.nifi.registry.serialization;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
+import org.apache.nifi.registry.flow.VersionedProcessor;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -38,13 +39,20 @@ public class TestFlowSnapshotSerializer {
         snapshotMetadata.setComments("This is the first flow");
         snapshotMetadata.setTimestamp(System.currentTimeMillis());
 
-        final VersionedProcessGroup processGroup = new VersionedProcessGroup();
-        processGroup.setIdentifier("pg1");
-        processGroup.setName("My Process Group");
+        final VersionedProcessGroup processGroup1 = new VersionedProcessGroup();
+        processGroup1.setIdentifier("pg1");
+        processGroup1.setName("My Process Group");
+
+        final VersionedProcessor processor1 = new VersionedProcessor();
+        processor1.setIdentifier("processor1");
+        processor1.setName("My Processor 1");
+
+        // make sure nested objects are serialized/deserialized
+        processGroup1.getProcessors().add(processor1);
 
         final VersionedFlowSnapshot snapshot = new VersionedFlowSnapshot();
         snapshot.setSnapshotMetadata(snapshotMetadata);
-        snapshot.setFlowContents(processGroup);
+        snapshot.setFlowContents(processGroup1);
 
         final ByteArrayOutputStream out = new ByteArrayOutputStream();
         serializer.serialize(snapshot, out);
@@ -52,7 +60,7 @@ public class TestFlowSnapshotSerializer {
         final ByteArrayInputStream in = new ByteArrayInputStream(out.toByteArray());
         final VersionedFlowSnapshot deserializedSnapshot = serializer.deserialize(in);
         final VersionedFlowSnapshotMetadata deserializedMetadata = deserializedSnapshot.getSnapshotMetadata();
-        final VersionedProcessGroup deserializedProcessGroup = deserializedSnapshot.getFlowContents();
+        final VersionedProcessGroup deserializedProcessGroup1 = deserializedSnapshot.getFlowContents();
 
         Assert.assertEquals(snapshotMetadata.getFlowIdentifier(), deserializedMetadata.getFlowIdentifier());
         Assert.assertEquals(snapshotMetadata.getFlowName(), deserializedMetadata.getFlowName());
@@ -60,7 +68,14 @@ public class TestFlowSnapshotSerializer {
         Assert.assertEquals(snapshotMetadata.getComments(), deserializedMetadata.getComments());
         Assert.assertEquals(snapshotMetadata.getTimestamp(), deserializedMetadata.getTimestamp());
 
-        Assert.assertEquals(processGroup.getIdentifier(), deserializedProcessGroup.getIdentifier());
-        Assert.assertEquals(processGroup.getName(), deserializedProcessGroup.getName());
+        Assert.assertEquals(processGroup1.getIdentifier(), deserializedProcessGroup1.getIdentifier());
+        Assert.assertEquals(processGroup1.getName(), deserializedProcessGroup1.getName());
+
+        Assert.assertEquals(1, deserializedProcessGroup1.getProcessors().size());
+
+        final VersionedProcessor deserializedProcessor1 = deserializedProcessGroup1.getProcessors().iterator().next();
+        Assert.assertEquals(processor1.getIdentifier(), deserializedProcessor1.getIdentifier());
+        Assert.assertEquals(processor1.getName(), deserializedProcessor1.getName());
+
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
index 50184dd..47a81cc 100644
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
+++ b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
@@ -123,6 +123,7 @@ public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvide
             new ResourceActionPair("/buckets", READ_CODE),
             new ResourceActionPair("/buckets", WRITE_CODE),
             new ResourceActionPair("/buckets", DELETE_CODE),
+            new ResourceActionPair("/proxy", WRITE_CODE)
     };
 
     static final String PROP_NODE_IDENTITY_PREFIX = "Node Identity ";

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-web-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/pom.xml b/nifi-registry-web-api/pom.xml
index 14fb0f4..0174bd5 100644
--- a/nifi-registry-web-api/pom.xml
+++ b/nifi-registry-web-api/pom.xml
@@ -211,10 +211,5 @@
             <version>0.0.1-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-data-model</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
-        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/071d08aa/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
index 013d9c9..c78e84d 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
@@ -267,7 +267,7 @@ public class AccessPolicyResource extends AuthorizableApplicationResource {
 
         authorizeAccessToPolicy(RequestAction.WRITE, identifier);
 
-        AccessPolicy createdPolicy = authorizationService.createAccessPolicy(requestAccessPolicy);
+        AccessPolicy createdPolicy = authorizationService.updateAccessPolicy(requestAccessPolicy);
 
         String locationUri = generateAccessPolicyUri(createdPolicy);
         return generateCreatedResponse(URI.create(locationUri), createdPolicy).build();


[2/2] nifi-registry git commit: NIFIREG-47 Fixing additional method that needed Bucket[].class instead of List.class

Posted by ma...@apache.org.
NIFIREG-47 Fixing additional method that needed Bucket[].class instead of List.class


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

Branch: refs/heads/master
Commit: a43e81fed3f55549ab1724fb6a6fc3dbe7d9b4ed
Parents: 071d08a
Author: Bryan Bende <bb...@apache.org>
Authored: Mon Nov 6 16:00:57 2017 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Nov 7 10:50:01 2017 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/registry/client/impl/JerseyBucketClient.java  | 3 ++-
 .../nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java   | 1 +
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/a43e81fe/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
index b1734a6..a971dff 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyBucketClient.java
@@ -150,7 +150,8 @@ public class JerseyBucketClient extends AbstractJerseyClient implements BucketCl
                 target = target.queryParam("sort", sortParam.toString());
             }
 
-            return getRequestBuilder(target).get(List.class);
+            final Bucket[] buckets = getRequestBuilder(target).get(Bucket[].class);
+            return buckets == null ? Collections.emptyList() : Arrays.asList(buckets);
         });
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/a43e81fe/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java b/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
index e813193..3892438 100644
--- a/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
+++ b/nifi-registry-client/src/test/java/org/apache/nifi/registry/client/impl/TestJerseyNiFiRegistryClient.java
@@ -99,6 +99,7 @@ public class TestJerseyNiFiRegistryClient {
             final List<Bucket> allBucketsSorted = bucketClient.getAll(Arrays.asList(sortParam));
             System.out.println("Retrieved sorted buckets, size = " + allBucketsSorted.size());
             Assert.assertEquals(numBuckets, allBucketsSorted.size());
+            allBucketsSorted.stream().forEach(b -> System.out.println("Retrieve bucket " + b.getIdentifier()));
 
             // update each bucket
             for (final Bucket bucket : createdBuckets) {