mirror of https://github.com/apache/nifi.git
NIFI-6112: Add some useful commands to NiFi Toolkit for automating NiFi cluster construction.
- Supports both UUID and identity for user / user group specification - Add UUID to toolkit command results - Fix representation for type and bundle of ControllerServiceResult - Make description more detailed - Fix to print only in interactive mode or verbose mode in UpdateAccessPolicy. (with slight refactoring) - Add list-templates command - Add start/stop commands for reporting task and enable/disable commands for reporting task controller service. - Also added "get-repoting-task" command (for single reporting task) to be consistent with controller service. This closes #3366. Signed-off-by: Bryan Bende <bbende@apache.org>
This commit is contained in:
parent
76392ee862
commit
3696b5bfcf
|
@ -61,6 +61,17 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-framework-core</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>ch.qos.logback</groupId>
|
||||
<artifactId>logback-classic</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi.registry</groupId>
|
||||
<artifactId>nifi-registry-client</artifactId>
|
||||
|
@ -86,5 +97,20 @@
|
|||
<artifactId>commons-io</artifactId>
|
||||
<version>2.6</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.glassfish.jersey.media</groupId>
|
||||
<artifactId>jersey-media-multipart</artifactId>
|
||||
<version>2.27</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.glassfish.jersey.media</groupId>
|
||||
<artifactId>jersey-media-jaxb</artifactId>
|
||||
<version>2.27</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.opencsv</groupId>
|
||||
<artifactId>opencsv</artifactId>
|
||||
<version>4.5</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* 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.toolkit.cli.api;
|
||||
|
||||
public enum AccessPolicyAction {
|
||||
|
||||
READ,
|
||||
WRITE;
|
||||
|
||||
}
|
|
@ -21,10 +21,15 @@ import org.apache.commons.lang3.StringUtils;
|
|||
import org.apache.nifi.registry.security.util.KeystoreType;
|
||||
import org.apache.nifi.toolkit.cli.api.ClientFactory;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.FlowClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientConfig;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TemplatesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.VersionsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.impl.JerseyNiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
|
@ -135,6 +140,21 @@ public class NiFiClientFactory implements ClientFactory<NiFiClient> {
|
|||
return wrappedClient.getControllerClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesClient getControllerServicesClient() {
|
||||
return wrappedClient.getControllerServicesClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesClient getControllerServicesClientForProxiedEntities(String... proxiedEntity) {
|
||||
return wrappedClient.getControllerServicesClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesClient getControllerServicesClientForToken(String token) {
|
||||
return wrappedClient.getControllerServicesClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowClient getFlowClient() {
|
||||
return wrappedClient.getFlowClientForProxiedEntities(proxiedEntity);
|
||||
|
@ -180,6 +200,66 @@ public class NiFiClientFactory implements ClientFactory<NiFiClient> {
|
|||
return wrappedClient.getVersionsClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TenantsClient getTenantsClient() {
|
||||
return wrappedClient.getTenantsClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TenantsClient getTenantsClientForProxiedEntities(String... proxiedEntity) {
|
||||
return wrappedClient.getTenantsClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TenantsClient getTenantsClientForToken(String token) {
|
||||
return wrappedClient.getTenantsClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PoliciesClient getPoliciesClient() {
|
||||
return wrappedClient.getPoliciesClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PoliciesClient getPoliciesClientForProxiedEntities(String... proxiedEntity) {
|
||||
return wrappedClient.getPoliciesClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PoliciesClient getPoliciesClientForToken(String token) {
|
||||
return wrappedClient.getPoliciesClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesClient getTemplatesClient() {
|
||||
return wrappedClient.getTemplatesClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesClient getTemplatesClientForProxiedEntities(String... proxiedEntity) {
|
||||
return wrappedClient.getTemplatesClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesClient getTemplatesClientForToken(String token) {
|
||||
return wrappedClient.getTemplatesClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksClient getReportingTasksClient() {
|
||||
return wrappedClient.getReportingTasksClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksClient getReportingTasksClientForProxiedEntities(String... proxiedEntity) {
|
||||
return wrappedClient.getReportingTasksClientForProxiedEntities(proxiedEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksClient getReportingTasksClientForToken(String token) {
|
||||
return wrappedClient.getReportingTasksClientForToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
wrappedClient.close();
|
||||
|
|
|
@ -17,9 +17,11 @@
|
|||
package org.apache.nifi.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.entity.ClusterEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.NodeEntity;
|
||||
import org.apache.nifi.web.api.entity.RegistryClientEntity;
|
||||
import org.apache.nifi.web.api.entity.RegistryClientsEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -48,4 +50,8 @@ public interface ControllerClient {
|
|||
|
||||
NodeEntity offloadNode(String nodeId, NodeEntity nodeEntity) throws NiFiClientException, IOException;
|
||||
|
||||
ControllerServiceEntity createControllerService(ControllerServiceEntity controllerService) throws NiFiClientException, IOException;
|
||||
|
||||
ReportingTaskEntity createReportingTask(ReportingTaskEntity reportingTask) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Client for interacting with NiFi's Controller Services Resource.
|
||||
*/
|
||||
public interface ControllerServicesClient {
|
||||
|
||||
ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
|
||||
|
||||
ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
|
@ -21,7 +21,9 @@ import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
|
|||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
import org.apache.nifi.web.api.entity.CurrentUserEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
|
||||
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
||||
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataSetEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -104,4 +106,25 @@ public interface FlowClient {
|
|||
*/
|
||||
ClusteSummaryEntity getClusterSummary() throws NiFiClientException, IOException;
|
||||
|
||||
/**
|
||||
* Retrieves the controller services for the reporting tasks.
|
||||
*
|
||||
* @return the controller services entity
|
||||
*/
|
||||
ControllerServicesEntity getControllerServices() throws NiFiClientException, IOException;
|
||||
|
||||
/**
|
||||
* Retrieves the reporting tasks.
|
||||
*
|
||||
* @return the reporting tasks entity
|
||||
*/
|
||||
ReportingTasksEntity getReportingTasks() throws NiFiClientException, IOException;
|
||||
|
||||
/**
|
||||
* Retrieves the all templates.
|
||||
*
|
||||
* @return the templates entity
|
||||
*/
|
||||
TemplatesEntity getTemplates() throws NiFiClientException, IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -48,6 +48,31 @@ public interface NiFiClient extends Closeable {
|
|||
*/
|
||||
ControllerClient getControllerClientForToken(String token);
|
||||
|
||||
// ----- ControllerServicesClient -----
|
||||
|
||||
/**
|
||||
* @return a ControllerServicesClient
|
||||
*/
|
||||
ControllerServicesClient getControllerServicesClient();
|
||||
|
||||
/**
|
||||
* Obtains a ControllerServicesClient for the given proxied entities. Each operation made from this client
|
||||
* will add the appropriate X-ProxiedEntitiesChain header to each request.
|
||||
*
|
||||
* @param proxiedEntity one or more identities to proxy
|
||||
* @return a ControllerServicesClient
|
||||
*/
|
||||
ControllerServicesClient getControllerServicesClientForProxiedEntities(String ... proxiedEntity);
|
||||
|
||||
/**
|
||||
* Obtains a ControllerServicesClient that will submit the given token in the Authorization Bearer header
|
||||
* with each request.
|
||||
*
|
||||
* @param token a token to authentication with
|
||||
* @return a ControllerServicesClient
|
||||
*/
|
||||
ControllerServicesClient getControllerServicesClientForToken(String token);
|
||||
|
||||
// ----- FlowClient -----
|
||||
|
||||
FlowClient getFlowClient();
|
||||
|
@ -72,6 +97,38 @@ public interface NiFiClient extends Closeable {
|
|||
|
||||
VersionsClient getVersionsClientForToken(String token);
|
||||
|
||||
// ----- TenantsClient -----
|
||||
|
||||
TenantsClient getTenantsClient();
|
||||
|
||||
TenantsClient getTenantsClientForProxiedEntities(String ... proxiedEntity);
|
||||
|
||||
TenantsClient getTenantsClientForToken(String token);
|
||||
|
||||
// ----- PoliciesClient -----
|
||||
|
||||
PoliciesClient getPoliciesClient();
|
||||
|
||||
PoliciesClient getPoliciesClientForProxiedEntities(String ... proxiedEntity);
|
||||
|
||||
PoliciesClient getPoliciesClientForToken(String token);
|
||||
|
||||
// ----- TemplatesClient -----
|
||||
|
||||
TemplatesClient getTemplatesClient();
|
||||
|
||||
TemplatesClient getTemplatesClientForProxiedEntities(String ... proxiedEntity);
|
||||
|
||||
TemplatesClient getTemplatesClientForToken(String token);
|
||||
|
||||
// ----- ReportingTasksClient -----
|
||||
|
||||
ReportingTasksClient getReportingTasksClient();
|
||||
|
||||
ReportingTasksClient getReportingTasksClientForProxiedEntities(String ... proxiedEntity);
|
||||
|
||||
ReportingTasksClient getReportingTasksClientForToken(String token);
|
||||
|
||||
/**
|
||||
* The builder interface that implementations should provide for obtaining the client.
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.entity.AccessPolicyEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface PoliciesClient {
|
||||
|
||||
AccessPolicyEntity getAccessPolicy(String resource, String action) throws NiFiClientException, IOException;
|
||||
|
||||
AccessPolicyEntity createAccessPolicy(AccessPolicyEntity accessPolicyEntity) throws NiFiClientException, IOException;
|
||||
|
||||
AccessPolicyEntity updateAccessPolicy(AccessPolicyEntity accessPolicyEntity) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
|
@ -16,7 +16,10 @@
|
|||
*/
|
||||
package org.apache.nifi.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||
import org.apache.nifi.web.api.entity.VariableRegistryEntity;
|
||||
import org.apache.nifi.web.api.entity.VariableRegistryUpdateRequestEntity;
|
||||
|
||||
|
@ -46,4 +49,9 @@ public interface ProcessGroupClient {
|
|||
VariableRegistryUpdateRequestEntity deleteVariableRegistryUpdateRequest(String processGroupdId, String requestId)
|
||||
throws NiFiClientException, IOException;
|
||||
|
||||
ControllerServiceEntity createControllerService(String processGroupId, ControllerServiceEntity controllerService)
|
||||
throws NiFiClientException, IOException;
|
||||
|
||||
TemplateEntity uploadTemplate(String processGroupId, TemplateDTO templateDTO) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Client for interacting with NiFi's Reporting Tasks Resource.
|
||||
*/
|
||||
public interface ReportingTasksClient {
|
||||
|
||||
ReportingTaskEntity getReportingTask(String id) throws NiFiClientException, IOException;
|
||||
|
||||
ReportingTaskEntity activateReportingTask(String id, ReportingTaskRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface TemplatesClient {
|
||||
|
||||
TemplateDTO getTemplate(String templateId) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi;
|
||||
|
||||
import org.apache.nifi.web.api.entity.UserEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupsEntity;
|
||||
import org.apache.nifi.web.api.entity.UsersEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface TenantsClient {
|
||||
|
||||
UsersEntity getUsers() throws NiFiClientException, IOException;
|
||||
|
||||
UserEntity createUser(UserEntity userEntity) throws NiFiClientException, IOException;
|
||||
|
||||
UserGroupEntity getUserGroup(String userGroupId) throws NiFiClientException, IOException;
|
||||
|
||||
UserGroupsEntity getUserGroups() throws NiFiClientException, IOException;
|
||||
|
||||
UserGroupEntity createUserGroup(UserGroupEntity userGroupEntity) throws NiFiClientException, IOException;
|
||||
|
||||
UserGroupEntity updateUserGroup(UserGroupEntity userGroupEntity) throws NiFiClientException, IOException;
|
||||
|
||||
}
|
|
@ -20,9 +20,11 @@ import org.apache.commons.lang3.StringUtils;
|
|||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.web.api.entity.ClusterEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.NodeEntity;
|
||||
import org.apache.nifi.web.api.entity.RegistryClientEntity;
|
||||
import org.apache.nifi.web.api.entity.RegistryClientsEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import javax.ws.rs.client.Entity;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
|
@ -191,4 +193,36 @@ public class JerseyControllerClient extends AbstractJerseyClient implements Cont
|
|||
return getRequestBuilder(target).get(ClusterEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity createControllerService(final ControllerServiceEntity controllerService) throws NiFiClientException, IOException {
|
||||
if (controllerService == null) {
|
||||
throw new IllegalArgumentException("Controller service entity cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error creating controller service", () -> {
|
||||
final WebTarget target = controllerTarget.path("controller-services");
|
||||
|
||||
return getRequestBuilder(target).post(
|
||||
Entity.entity(controllerService, MediaType.APPLICATION_JSON),
|
||||
ControllerServiceEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskEntity createReportingTask(ReportingTaskEntity reportingTask) throws NiFiClientException, IOException {
|
||||
if (reportingTask == null) {
|
||||
throw new IllegalArgumentException("Reporting task entity cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error creating reporting task", () -> {
|
||||
final WebTarget target = controllerTarget.path("reporting-tasks");
|
||||
|
||||
return getRequestBuilder(target).post(
|
||||
Entity.entity(reportingTask, MediaType.APPLICATION_JSON),
|
||||
ReportingTaskEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi.impl;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import javax.ws.rs.client.Entity;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
|
||||
/**
|
||||
* Jersey implementation of ControllerServicersClient.
|
||||
*/
|
||||
public class JerseyControllerServicesClient extends AbstractJerseyClient implements ControllerServicesClient {
|
||||
|
||||
private final WebTarget controllerServicesTarget;
|
||||
|
||||
public JerseyControllerServicesClient(final WebTarget baseTarget) {
|
||||
this(baseTarget, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public JerseyControllerServicesClient(final WebTarget baseTarget, final Map<String, String> headers) {
|
||||
super(headers);
|
||||
this.controllerServicesTarget = baseTarget.path("/controller-services");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity getControllerService(final String id) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(id)) {
|
||||
throw new IllegalArgumentException("Controller service id cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error retrieving status of controller service", () -> {
|
||||
final WebTarget target = controllerServicesTarget.path("{id}").resolveTemplate("id", id);
|
||||
return getRequestBuilder(target).get(ControllerServiceEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity activateControllerService(final String id,
|
||||
final ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(id)) {
|
||||
throw new IllegalArgumentException("Controller service id cannot be null");
|
||||
}
|
||||
|
||||
if (runStatusEntity == null) {
|
||||
throw new IllegalArgumentException("Entity cannnot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error enabling or disabling controller service", () -> {
|
||||
final WebTarget target = controllerServicesTarget
|
||||
.path("{id}/run-status").resolveTemplate("id", id);
|
||||
return getRequestBuilder(target).put(
|
||||
Entity.entity(runStatusEntity, MediaType.APPLICATION_JSON_TYPE),
|
||||
ControllerServiceEntity.class);
|
||||
});
|
||||
}
|
||||
}
|
|
@ -29,7 +29,9 @@ import org.apache.nifi.web.api.entity.ComponentEntity;
|
|||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
import org.apache.nifi.web.api.entity.CurrentUserEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
|
||||
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
||||
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataSetEntity;
|
||||
|
||||
import javax.ws.rs.client.Entity;
|
||||
|
@ -226,4 +228,28 @@ public class JerseyFlowClient extends AbstractJerseyClient implements FlowClient
|
|||
return getRequestBuilder(target).get(ClusteSummaryEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesEntity getControllerServices() throws NiFiClientException, IOException {
|
||||
return executeAction("Error retrieving reporting task controller services", () -> {
|
||||
final WebTarget target = flowTarget.path("controller/controller-services");
|
||||
return getRequestBuilder(target).get(ControllerServicesEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksEntity getReportingTasks() throws NiFiClientException, IOException {
|
||||
return executeAction("Error retrieving reporting tasks", () -> {
|
||||
final WebTarget target = flowTarget.path("reporting-tasks");
|
||||
return getRequestBuilder(target).get(ReportingTasksEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesEntity getTemplates() throws NiFiClientException, IOException {
|
||||
return executeAction("Error retrieving templates", () -> {
|
||||
final WebTarget target = flowTarget.path("templates");
|
||||
return getRequestBuilder(target).get(TemplatesEntity.class);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,10 +23,15 @@ import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.FlowClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientConfig;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TemplatesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.VersionsClient;
|
||||
import org.glassfish.jersey.client.ClientConfig;
|
||||
import org.glassfish.jersey.client.ClientProperties;
|
||||
|
@ -126,6 +131,23 @@ public class JerseyNiFiClient implements NiFiClient {
|
|||
return new JerseyControllerClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesClient getControllerServicesClient() {
|
||||
return new JerseyControllerServicesClient(baseTarget);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesClient getControllerServicesClientForProxiedEntities(final String... proxiedEntity) {
|
||||
final Map<String, String> headers = getHeaders(proxiedEntity);
|
||||
return new JerseyControllerServicesClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesClient getControllerServicesClientForToken(final String base64token) {
|
||||
final Map<String, String> headers = getHeadersWithToken(base64token);
|
||||
return new JerseyControllerServicesClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowClient getFlowClient() {
|
||||
return new JerseyFlowClient(baseTarget);
|
||||
|
@ -177,6 +199,74 @@ public class JerseyNiFiClient implements NiFiClient {
|
|||
return new JerseyVersionsClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TenantsClient getTenantsClient() {
|
||||
return new JerseyTenantsClient(baseTarget);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TenantsClient getTenantsClientForProxiedEntities(String... proxiedEntity) {
|
||||
final Map<String, String> headers = getHeaders(proxiedEntity);
|
||||
return new JerseyTenantsClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TenantsClient getTenantsClientForToken(String base64token) {
|
||||
final Map<String, String> headers = getHeadersWithToken(base64token);
|
||||
return new JerseyTenantsClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PoliciesClient getPoliciesClient() {
|
||||
return new JerseyPoliciesClient(baseTarget);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PoliciesClient getPoliciesClientForProxiedEntities(String... proxiedEntity) {
|
||||
final Map<String, String> headers = getHeaders(proxiedEntity);
|
||||
return new JerseyPoliciesClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PoliciesClient getPoliciesClientForToken(String base64token) {
|
||||
final Map<String, String> headers = getHeadersWithToken(base64token);
|
||||
return new JerseyPoliciesClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesClient getTemplatesClient() {
|
||||
return new JerseyTemplatesClient(baseTarget);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesClient getTemplatesClientForProxiedEntities(String... proxiedEntity) {
|
||||
final Map<String, String> headers = getHeaders(proxiedEntity);
|
||||
return new JerseyTemplatesClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesClient getTemplatesClientForToken(String base64token) {
|
||||
final Map<String, String> headers = getHeadersWithToken(base64token);
|
||||
return new JerseyTemplatesClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksClient getReportingTasksClient() {
|
||||
return new JerseyReportingTasksClient(baseTarget);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksClient getReportingTasksClientForProxiedEntities(String... proxiedEntity) {
|
||||
final Map<String, String> headers = getHeaders(proxiedEntity);
|
||||
return new JerseyReportingTasksClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksClient getReportingTasksClientForToken(String base64token) {
|
||||
final Map<String, String> headers = getHeadersWithToken(base64token);
|
||||
return new JerseyReportingTasksClient(baseTarget, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.client != null) {
|
||||
|
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi.impl;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
|
||||
import org.apache.nifi.web.api.entity.AccessPolicyEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import javax.ws.rs.client.Entity;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
|
||||
/**
|
||||
* Jersey implementation of PoliciesClient.
|
||||
*/
|
||||
public class JerseyPoliciesClient extends AbstractJerseyClient implements PoliciesClient {
|
||||
|
||||
private final WebTarget policiesTarget;
|
||||
|
||||
public JerseyPoliciesClient(final WebTarget baseTarget) {
|
||||
this(baseTarget, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public JerseyPoliciesClient(final WebTarget baseTarget, final Map<String, String> headers) {
|
||||
super(headers);
|
||||
this.policiesTarget = baseTarget.path("/policies");
|
||||
}
|
||||
|
||||
@Override
|
||||
public AccessPolicyEntity getAccessPolicy(final String resource, final String action) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(resource) || StringUtils.isBlank(action)) {
|
||||
throw new IllegalArgumentException("Resouce and action cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error retrieving configuration of access policy", () -> {
|
||||
final WebTarget target = policiesTarget.path(action).path(resource);
|
||||
return getRequestBuilder(target).get(AccessPolicyEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public AccessPolicyEntity createAccessPolicy(final AccessPolicyEntity accessPolicyEntity) throws NiFiClientException, IOException {
|
||||
if (accessPolicyEntity == null) {
|
||||
throw new IllegalArgumentException("Access policy entity cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error creating access policy", () ->
|
||||
getRequestBuilder(policiesTarget).post(
|
||||
Entity.entity(accessPolicyEntity, MediaType.APPLICATION_JSON),
|
||||
AccessPolicyEntity.class
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AccessPolicyEntity updateAccessPolicy(final AccessPolicyEntity accessPolicyEntity) throws NiFiClientException, IOException {
|
||||
if (accessPolicyEntity == null) {
|
||||
throw new IllegalArgumentException("Access policy entity cannot be null");
|
||||
}
|
||||
|
||||
if (StringUtils.isBlank(accessPolicyEntity.getId())) {
|
||||
throw new IllegalArgumentException("Access policy entity must contain an id");
|
||||
}
|
||||
|
||||
return executeAction("Error updating access policy", () -> {
|
||||
final WebTarget target = policiesTarget.path(accessPolicyEntity.getId());
|
||||
|
||||
return getRequestBuilder(target).put(
|
||||
Entity.entity(accessPolicyEntity, MediaType.APPLICATION_JSON),
|
||||
AccessPolicyEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
}
|
|
@ -19,9 +19,14 @@ package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||
import org.apache.nifi.web.api.entity.VariableRegistryEntity;
|
||||
import org.apache.nifi.web.api.entity.VariableRegistryUpdateRequestEntity;
|
||||
import org.glassfish.jersey.media.multipart.FormDataMultiPart;
|
||||
import org.glassfish.jersey.media.multipart.MultiPartFeature;
|
||||
|
||||
import javax.ws.rs.client.Entity;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
|
@ -187,4 +192,54 @@ public class JerseyProcessGroupClient extends AbstractJerseyClient implements Pr
|
|||
return getRequestBuilder(target).delete(VariableRegistryUpdateRequestEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity createControllerService(
|
||||
final String processGroupId, final ControllerServiceEntity controllerService) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(processGroupId)) {
|
||||
throw new IllegalArgumentException("Process group id cannot be null or blank");
|
||||
}
|
||||
|
||||
if (controllerService == null) {
|
||||
throw new IllegalArgumentException("Controller service entity cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error creating controller service", () -> {
|
||||
final WebTarget target = processGroupsTarget
|
||||
.path("{id}/controller-services")
|
||||
.resolveTemplate("id", processGroupId);
|
||||
|
||||
return getRequestBuilder(target).post(
|
||||
Entity.entity(controllerService, MediaType.APPLICATION_JSON),
|
||||
ControllerServiceEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplateEntity uploadTemplate(
|
||||
final String processGroupId, final TemplateDTO templateDTO) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(processGroupId)) {
|
||||
throw new IllegalArgumentException("Process group id cannot be null or blank");
|
||||
}
|
||||
|
||||
if (templateDTO == null) {
|
||||
throw new IllegalArgumentException("The template dto cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error uploading template file", () -> {
|
||||
final WebTarget target = processGroupsTarget
|
||||
.path("{id}/templates/upload")
|
||||
.resolveTemplate("id", processGroupId)
|
||||
.register(MultiPartFeature.class);
|
||||
|
||||
FormDataMultiPart form = new FormDataMultiPart();
|
||||
form.field("template", templateDTO, MediaType.TEXT_XML_TYPE);
|
||||
|
||||
return getRequestBuilder(target).post(
|
||||
Entity.entity(form, MediaType.MULTIPART_FORM_DATA),
|
||||
TemplateEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import javax.ws.rs.client.Entity;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
|
||||
/**
|
||||
* Jersey implementation of ReportingTasksClient.
|
||||
*/
|
||||
public class JerseyReportingTasksClient extends AbstractJerseyClient implements ReportingTasksClient {
|
||||
|
||||
private final WebTarget reportingTasksTarget;
|
||||
|
||||
public JerseyReportingTasksClient(final WebTarget baseTarget) {
|
||||
this(baseTarget, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public JerseyReportingTasksClient(final WebTarget baseTarget, final Map<String, String> headers) {
|
||||
super(headers);
|
||||
this.reportingTasksTarget = baseTarget.path("/reporting-tasks");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskEntity getReportingTask(final String id) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(id)) {
|
||||
throw new IllegalArgumentException("Reporting task id cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error retrieving status of reporting task", () -> {
|
||||
final WebTarget target = reportingTasksTarget.path(id);
|
||||
return getRequestBuilder(target).get(ReportingTaskEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskEntity activateReportingTask(final String id,
|
||||
final ReportingTaskRunStatusEntity runStatusEntity) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(id)) {
|
||||
throw new IllegalArgumentException("Reporting task id cannot be null");
|
||||
}
|
||||
|
||||
if (runStatusEntity == null) {
|
||||
throw new IllegalArgumentException("Entity cannnot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error starting or stopping report task", () -> {
|
||||
final WebTarget target = reportingTasksTarget
|
||||
.path("{id}/run-status").resolveTemplate("id", id);
|
||||
return getRequestBuilder(target).put(
|
||||
Entity.entity(runStatusEntity, MediaType.APPLICATION_JSON_TYPE),
|
||||
ReportingTaskEntity.class);
|
||||
});
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi.impl;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TemplatesClient;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
|
||||
/**
|
||||
* Jersey implementation of TemplatesClient.
|
||||
*/
|
||||
public class JerseyTemplatesClient extends AbstractJerseyClient implements TemplatesClient {
|
||||
|
||||
private final WebTarget templatesTarget;
|
||||
|
||||
public JerseyTemplatesClient(final WebTarget baseTarget) {
|
||||
this(baseTarget, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public JerseyTemplatesClient(final WebTarget baseTarget, final Map<String, String> headers) {
|
||||
super(headers);
|
||||
this.templatesTarget = baseTarget.path("/templates");
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplateDTO getTemplate(final String templateId) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(templateId)) {
|
||||
throw new IllegalArgumentException("Template id cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error retrieving template", () -> {
|
||||
final WebTarget target = templatesTarget
|
||||
.path("{id}/download")
|
||||
.resolveTemplate("id", templateId);
|
||||
return getRequestBuilder(target).get(TemplateDTO.class);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,133 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.client.nifi.impl;
|
||||
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.util.StringUtils;
|
||||
import org.apache.nifi.web.api.entity.UserEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupsEntity;
|
||||
import org.apache.nifi.web.api.entity.UsersEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import javax.ws.rs.client.Entity;
|
||||
import javax.ws.rs.client.WebTarget;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
|
||||
/**
|
||||
* Jersey implementation of TenantsClient.
|
||||
*/
|
||||
public class JerseyTenantsClient extends AbstractJerseyClient implements TenantsClient {
|
||||
|
||||
private final WebTarget tenantsTarget;
|
||||
|
||||
public JerseyTenantsClient(final WebTarget baseTarget) {
|
||||
this(baseTarget, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public JerseyTenantsClient(final WebTarget baseTarget, final Map<String, String> headers) {
|
||||
super(headers);
|
||||
this.tenantsTarget = baseTarget.path("/tenants");
|
||||
}
|
||||
|
||||
@Override
|
||||
public UsersEntity getUsers() throws NiFiClientException, IOException {
|
||||
return executeAction("Error retrieving users", () -> {
|
||||
final WebTarget target = tenantsTarget.path("users");
|
||||
return getRequestBuilder(target).get(UsersEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserEntity createUser(final UserEntity userEntity) throws NiFiClientException, IOException {
|
||||
if (userEntity == null) {
|
||||
throw new IllegalArgumentException("User entity cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error creating user", () -> {
|
||||
final WebTarget target = tenantsTarget.path("users");
|
||||
|
||||
return getRequestBuilder(target).post(
|
||||
Entity.entity(userEntity, MediaType.APPLICATION_JSON),
|
||||
UserEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserGroupEntity getUserGroup(final String id) throws NiFiClientException, IOException {
|
||||
if (StringUtils.isBlank(id)) {
|
||||
throw new IllegalArgumentException("User group id cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error retrieving user group", () -> {
|
||||
final WebTarget target = tenantsTarget
|
||||
.path("user-groups/{id}")
|
||||
.resolveTemplate("id", id);
|
||||
return getRequestBuilder(target).get(UserGroupEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserGroupsEntity getUserGroups() throws NiFiClientException, IOException {
|
||||
return executeAction("Error retrieving user groups", () -> {
|
||||
final WebTarget target = tenantsTarget.path("user-groups");
|
||||
return getRequestBuilder(target).get(UserGroupsEntity.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserGroupEntity createUserGroup(final UserGroupEntity userGroupEntity) throws NiFiClientException, IOException {
|
||||
if (userGroupEntity == null) {
|
||||
throw new IllegalArgumentException("User group entity cannot be null");
|
||||
}
|
||||
|
||||
return executeAction("Error creating user group", () -> {
|
||||
final WebTarget target = tenantsTarget.path("user-groups");
|
||||
|
||||
return getRequestBuilder(target).post(
|
||||
Entity.entity(userGroupEntity, MediaType.APPLICATION_JSON),
|
||||
UserGroupEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserGroupEntity updateUserGroup(UserGroupEntity userGroupEntity) throws NiFiClientException, IOException {
|
||||
if (userGroupEntity == null) {
|
||||
throw new IllegalArgumentException("User group entity cannot be null");
|
||||
}
|
||||
|
||||
if (StringUtils.isBlank(userGroupEntity.getId())) {
|
||||
throw new IllegalArgumentException("User group entity must contain an id");
|
||||
}
|
||||
|
||||
return executeAction("Error updating user group", () -> {
|
||||
final WebTarget target = tenantsTarget
|
||||
.path("user-groups/{id}")
|
||||
.resolveTemplate("id", userGroupEntity.getId());
|
||||
|
||||
return getRequestBuilder(target).put(
|
||||
Entity.entity(userGroupEntity, MediaType.APPLICATION_JSON),
|
||||
UserGroupEntity.class
|
||||
);
|
||||
});
|
||||
}
|
||||
}
|
|
@ -64,6 +64,29 @@ public enum CommandOption {
|
|||
PG_VAR_NAME("var", "varName", "The name of a variable", true),
|
||||
PG_VAR_VALUE("val", "varValue", "The value of a variable", true),
|
||||
|
||||
// NiFi - Controller Services
|
||||
CS_ID("cs", "controllerServiceId", "The id of a controller service", true),
|
||||
|
||||
// NiFi - Reporting Tasks
|
||||
RT_ID("rt", "reportingTaskId", "The id of a reporting task", true),
|
||||
|
||||
// NiFi - User/Group
|
||||
USER_NAME("un", "userName", "The name of a user", true),
|
||||
UG_ID("ugid", "userGroupId", "The id of a user group", true),
|
||||
UG_NAME("ugn", "userGroupName", "The name of a user group", true),
|
||||
USER_NAME_LIST("unl", "userNameList", "The comma-separated user name list", true),
|
||||
USER_ID_LIST("uil", "userIdList", "The comma-separated user id list", true),
|
||||
GROUP_NAME_LIST("gnl", "groupNameList", "The comma-separated user group name list", true),
|
||||
GROUP_ID_LIST("gil", "groupIdList", "The comma-separated user group id list", true),
|
||||
|
||||
// NiFi - Access Policies
|
||||
POLICY_RESOURCE("por", "accessPolicyResource", "The resource of an access policy", true),
|
||||
POLICY_ACTION("poa", "accessPolicyAction", "The action of an access policy (read or write)", true),
|
||||
OVERWRITE_POLICY("owp", "overwritePolicy", "Overwrite the user list and group list for the access policy", false),
|
||||
|
||||
// NiFi - Templates
|
||||
TEMPLATE_ID("tid", "templateId", "The id of a template", true),
|
||||
|
||||
// Security related
|
||||
KEYSTORE("ks", "keystore", "A keystore to use for TLS/SSL connections", true),
|
||||
KEYSTORE_TYPE("kst", "keystoreType", "The type of key store being used (JKS or PKCS12)", true),
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi;
|
||||
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.entity.ComponentEntity;
|
||||
import org.apache.nifi.web.api.entity.ComponentRunStatusEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Base class for NiFi compornent activation commands.
|
||||
*/
|
||||
public abstract class AbstractNiFiActivateCommand<C extends ComponentEntity, S extends ComponentRunStatusEntity>
|
||||
extends AbstractNiFiCommand<VoidResult> {
|
||||
|
||||
public AbstractNiFiActivateCommand(final String name) {
|
||||
super(name, VoidResult.class);
|
||||
}
|
||||
|
||||
protected void activate(final NiFiClient client, final Properties properties,
|
||||
final Set<C> componentEntities, final String state) throws IOException, CommandException {
|
||||
if (shouldPrint(properties)) {
|
||||
println();
|
||||
}
|
||||
|
||||
final List<NiFiClientException> exceptions = new ArrayList<>();
|
||||
|
||||
for (final C componentEntity : componentEntities) {
|
||||
final RevisionDTO revisionDTO = new RevisionDTO();
|
||||
revisionDTO.setVersion(componentEntity.getRevision().getVersion());
|
||||
revisionDTO.setClientId(getContext().getSession().getNiFiClientID());
|
||||
|
||||
final S runStatusEntity = getRunStatusEntity();
|
||||
runStatusEntity.setRevision(revisionDTO);
|
||||
runStatusEntity.setState(state);
|
||||
runStatusEntity.validateState();
|
||||
|
||||
try {
|
||||
final C activated = activateComponent(client, componentEntity, runStatusEntity);
|
||||
|
||||
if (shouldPrint(properties)) {
|
||||
println(getDispName(activated) + " has been " + state.toLowerCase());
|
||||
println();
|
||||
}
|
||||
} catch (NiFiClientException e) {
|
||||
exceptions.add(e);
|
||||
|
||||
if (shouldPrint(properties)) {
|
||||
println(getDispName(componentEntity) + " could not be " + state.toLowerCase());
|
||||
println();
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (exceptions.size() > 0) {
|
||||
throw new CommandException(exceptions.size() + " components could not be " + state.toLowerCase() + ", " +
|
||||
"run command with -verbose to obtain more details");
|
||||
}
|
||||
}
|
||||
|
||||
private boolean shouldPrint(final Properties properties) {
|
||||
return isInteractive() || isVerbose(properties);
|
||||
}
|
||||
|
||||
public abstract S getRunStatusEntity();
|
||||
|
||||
public abstract C activateComponent(final NiFiClient client, final C componentEntity, final S runStatusEntity)
|
||||
throws NiFiClientException, IOException;
|
||||
|
||||
public abstract String getDispName(final C componentEntity);
|
||||
}
|
|
@ -16,7 +16,9 @@
|
|||
*/
|
||||
package org.apache.nifi.toolkit.cli.impl.command.nifi;
|
||||
|
||||
import com.opencsv.CSVParser;
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.ClientFactory;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Result;
|
||||
|
@ -25,9 +27,20 @@ import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
|||
import org.apache.nifi.toolkit.cli.impl.command.AbstractPropertyCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.session.SessionVariable;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.TenantDTO;
|
||||
import org.apache.nifi.web.api.entity.TenantEntity;
|
||||
import org.apache.nifi.web.api.entity.UserEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupsEntity;
|
||||
import org.apache.nifi.web.api.entity.UsersEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Base class for all NiFi commands.
|
||||
|
@ -73,4 +86,68 @@ public abstract class AbstractNiFiCommand<R extends Result> extends AbstractProp
|
|||
return revisionDTO;
|
||||
}
|
||||
|
||||
protected static Set<TenantEntity> generateTenantEntities(final String ids)
|
||||
throws IOException {
|
||||
final CSVParser csvParser = new CSVParser();
|
||||
return Arrays.stream(csvParser.parseLine(ids))
|
||||
.map(AbstractNiFiCommand::createTenantEntity)
|
||||
.collect(Collectors.toCollection(LinkedHashSet::new));
|
||||
}
|
||||
|
||||
protected static Set<TenantEntity> generateTenantEntities(final String users, final UsersEntity existingUsers)
|
||||
throws IOException, CommandException {
|
||||
final CSVParser csvParser = new CSVParser();
|
||||
final String[] userArray = csvParser.parseLine(users);
|
||||
final Set<TenantEntity> tenantEntities = new LinkedHashSet<>();
|
||||
|
||||
for (String user : userArray) {
|
||||
Optional<UserEntity> existingUser = existingUsers.getUsers().stream()
|
||||
.filter(entity -> user.equals(entity.getComponent().getIdentity())).findAny();
|
||||
|
||||
if (!existingUser.isPresent()) {
|
||||
throw new CommandException("User with the identity '" + user + "' not found.");
|
||||
}
|
||||
|
||||
tenantEntities.add(createTenantEntity(existingUser.get().getId(), user));
|
||||
}
|
||||
|
||||
return tenantEntities;
|
||||
}
|
||||
|
||||
protected static Set<TenantEntity> generateTenantEntities(final String groups, final UserGroupsEntity existingGroups)
|
||||
throws IOException, CommandException {
|
||||
final CSVParser csvParser = new CSVParser();
|
||||
final String[] groupArray = csvParser.parseLine(groups);
|
||||
final Set<TenantEntity> tenantEntities = new LinkedHashSet<>();
|
||||
|
||||
for (String group : groupArray) {
|
||||
Optional<UserGroupEntity> existingGroup = existingGroups.getUserGroups().stream()
|
||||
.filter(entity -> group.equals(entity.getComponent().getIdentity())).findAny();
|
||||
|
||||
if (!existingGroup.isPresent()) {
|
||||
throw new CommandException("User group with the identity '" + group + "' not found.");
|
||||
}
|
||||
|
||||
tenantEntities.add(createTenantEntity(existingGroup.get().getId(), group));
|
||||
}
|
||||
|
||||
return tenantEntities;
|
||||
}
|
||||
|
||||
private static TenantEntity createTenantEntity(final String id) {
|
||||
return createTenantEntity(id, null);
|
||||
}
|
||||
|
||||
private static TenantEntity createTenantEntity(final String id, final String identity) {
|
||||
TenantEntity tenantEntity = new TenantEntity();
|
||||
tenantEntity.setId(id);
|
||||
|
||||
if (StringUtils.isNotBlank(identity)) {
|
||||
TenantDTO tenantDTO = new TenantDTO();
|
||||
tenantDTO.setIdentity(identity);
|
||||
tenantEntity.setComponent(tenantDTO);
|
||||
}
|
||||
|
||||
return tenantEntity;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,9 +18,19 @@ package org.apache.nifi.toolkit.cli.impl.command.nifi;
|
|||
|
||||
import org.apache.nifi.toolkit.cli.api.Command;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.AbstractCommandGroup;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.cs.DisableControllerServices;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.cs.EnableControllerServices;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.ClusterSummary;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.cs.CreateControllerService;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.CreateReportingTask;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.CurrentUser;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.cs.GetControllerService;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.cs.GetControllerServices;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.GetReportingTask;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.GetReportingTasks;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.GetRootId;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.StartReportingTasks;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.StopReportingTasks;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.ConnectNode;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.OffloadNode;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.DeleteNode;
|
||||
|
@ -28,6 +38,7 @@ import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.DisconnectNode;
|
|||
import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.GetNode;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.GetNodes;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGChangeVersion;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGCreateControllerService;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGDisableControllerServices;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGEnableControllerServices;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGGetAllVersions;
|
||||
|
@ -40,10 +51,20 @@ import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGSetVar;
|
|||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGStart;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGStatus;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGStop;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.policies.GetAccessPolicy;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.policies.UpdateAccessPolicy;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.registry.CreateRegistryClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.registry.GetRegistryClientId;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.registry.ListRegistryClients;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.registry.UpdateRegistryClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.templates.DownloadTemplate;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.templates.ListTemplates;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.templates.UploadTemplate;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.tenants.CreateUser;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.tenants.CreateUserGroup;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.tenants.ListUserGroups;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.tenants.ListUsers;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.tenants.UpdateUserGroup;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -86,8 +107,29 @@ public class NiFiCommandGroup extends AbstractCommandGroup {
|
|||
commands.add(new PGList());
|
||||
commands.add(new PGStatus());
|
||||
commands.add(new PGGetControllerServices());
|
||||
commands.add(new PGCreateControllerService());
|
||||
commands.add(new PGEnableControllerServices());
|
||||
commands.add(new PGDisableControllerServices());
|
||||
commands.add(new GetControllerServices());
|
||||
commands.add(new GetControllerService());
|
||||
commands.add(new CreateControllerService());
|
||||
commands.add(new EnableControllerServices());
|
||||
commands.add(new DisableControllerServices());
|
||||
commands.add(new GetReportingTasks());
|
||||
commands.add(new GetReportingTask());
|
||||
commands.add(new CreateReportingTask());
|
||||
commands.add(new StartReportingTasks());
|
||||
commands.add(new StopReportingTasks());
|
||||
commands.add(new ListUsers());
|
||||
commands.add(new CreateUser());
|
||||
commands.add(new ListUserGroups());
|
||||
commands.add(new CreateUserGroup());
|
||||
commands.add(new UpdateUserGroup());
|
||||
commands.add(new GetAccessPolicy());
|
||||
commands.add(new UpdateAccessPolicy());
|
||||
commands.add(new ListTemplates());
|
||||
commands.add(new DownloadTemplate());
|
||||
commands.add(new UploadTemplate());
|
||||
return new ArrayList<>(commands);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.cs;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.StringResult;
|
||||
import org.apache.nifi.toolkit.cli.impl.util.JacksonUtils;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for creating a controller service for reporting tasks.
|
||||
*/
|
||||
public class CreateControllerService extends AbstractNiFiCommand<StringResult> {
|
||||
|
||||
public CreateControllerService() {
|
||||
super("create-service", StringResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Creates a controller service for reporting tasks from a local file.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.INPUT_SOURCE.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String inputFile = getRequiredArg(properties, CommandOption.INPUT_SOURCE);
|
||||
final URI uri = Paths.get(inputFile).toAbsolutePath().toUri();
|
||||
final String contents = IOUtils.toString(uri, StandardCharsets.UTF_8);
|
||||
|
||||
final ObjectMapper objectMapper = JacksonUtils.getObjectMapper();
|
||||
final ControllerServiceEntity deserializedService = objectMapper.readValue(contents, ControllerServiceEntity.class);
|
||||
if (deserializedService == null) {
|
||||
throw new IOException("Unable to deserialize controller service version from " + inputFile);
|
||||
}
|
||||
|
||||
deserializedService.setRevision(getInitialRevisionDTO());
|
||||
|
||||
final ControllerClient controllerClient = client.getControllerClient();
|
||||
final ControllerServiceEntity createdEntity = controllerClient.createControllerService(deserializedService);
|
||||
|
||||
return new StringResult(String.valueOf(createdEntity.getId()), getContext().isInteractive());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.cs;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiActivateCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command for disabling controller services for reporting tasks.
|
||||
*/
|
||||
public class DisableControllerServices extends AbstractNiFiActivateCommand<ControllerServiceEntity,
|
||||
ControllerServiceRunStatusEntity> {
|
||||
|
||||
public DisableControllerServices() {
|
||||
super("disable-services");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Disables all controller services for reporting tasks. Any services that are in use by a running " +
|
||||
"reporting task will fail to be disabled and will need to be stopped first using stop-reporting-tasks. " +
|
||||
"In stand-alone mode this command will not produce all of the output seen in interactive mode unless " +
|
||||
"the --verbose argument is specified.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.CS_ID.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VoidResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String csId = getArg(properties, CommandOption.CS_ID);
|
||||
final Set<ControllerServiceEntity> serviceEntities = new HashSet<>();
|
||||
|
||||
if (StringUtils.isBlank(csId)) {
|
||||
final ControllerServicesEntity servicesEntity = client.getFlowClient().getControllerServices();
|
||||
serviceEntities.addAll(servicesEntity.getControllerServices());
|
||||
} else {
|
||||
serviceEntities.add(client.getControllerServicesClient().getControllerService(csId));
|
||||
}
|
||||
|
||||
activate(client, properties, serviceEntities, "DISABLED");
|
||||
|
||||
return VoidResult.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceRunStatusEntity getRunStatusEntity() {
|
||||
return new ControllerServiceRunStatusEntity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity activateComponent(final NiFiClient client,
|
||||
final ControllerServiceEntity serviceEntity, final ControllerServiceRunStatusEntity runStatusEntity)
|
||||
throws NiFiClientException, IOException {
|
||||
return client.getControllerServicesClient().activateControllerService(serviceEntity.getId(), runStatusEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDispName(final ControllerServiceEntity serviceEntity) {
|
||||
return "Controller service \"" + serviceEntity.getComponent().getName() + "\" " +
|
||||
"(id: " + serviceEntity.getId() + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.cs;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiActivateCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command for enabling controller services for reporting tasks.
|
||||
*/
|
||||
public class EnableControllerServices extends AbstractNiFiActivateCommand<ControllerServiceEntity,
|
||||
ControllerServiceRunStatusEntity> {
|
||||
|
||||
public EnableControllerServices() {
|
||||
super("enable-services");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Attempts to enable all controller services for reporting tasks. In stand-alone mode this command " +
|
||||
"will not produce all of the output seen in interactive mode unless the --verbose argument is specified.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.CS_ID.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VoidResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String csId = getArg(properties, CommandOption.CS_ID);
|
||||
final Set<ControllerServiceEntity> serviceEntities = new HashSet<>();
|
||||
|
||||
if (StringUtils.isBlank(csId)) {
|
||||
final ControllerServicesEntity servicesEntity = client.getFlowClient().getControllerServices();
|
||||
serviceEntities.addAll(servicesEntity.getControllerServices());
|
||||
} else {
|
||||
serviceEntities.add(client.getControllerServicesClient().getControllerService(csId));
|
||||
}
|
||||
|
||||
activate(client, properties, serviceEntities, "ENABLED");
|
||||
|
||||
return VoidResult.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceRunStatusEntity getRunStatusEntity() {
|
||||
return new ControllerServiceRunStatusEntity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity activateComponent(final NiFiClient client,
|
||||
final ControllerServiceEntity serviceEntity, final ControllerServiceRunStatusEntity runStatusEntity)
|
||||
throws NiFiClientException, IOException {
|
||||
return client.getControllerServicesClient().activateControllerService(serviceEntity.getId(), runStatusEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDispName(final ControllerServiceEntity serviceEntity) {
|
||||
return "Controller service \"" + serviceEntity.getComponent().getName() + "\" " +
|
||||
"(id: " + serviceEntity.getId() + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.cs;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.ControllerServiceResult;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for retrieving the status of a controller service.
|
||||
*/
|
||||
public class GetControllerService extends AbstractNiFiCommand<ControllerServiceResult> {
|
||||
|
||||
public GetControllerService() {
|
||||
super("get-service", ControllerServiceResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the status for a controller service.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(Context context) {
|
||||
addOption(CommandOption.CS_ID.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException {
|
||||
final String csId = getRequiredArg(properties, CommandOption.CS_ID);
|
||||
final ControllerServicesClient csClient = client.getControllerServicesClient();
|
||||
|
||||
final ControllerServiceEntity csEntityResult = csClient.getControllerService(csId);
|
||||
return new ControllerServiceResult(getResultType(properties), csEntityResult);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.cs;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.FlowClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.ControllerServicesResult;
|
||||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to get the list of controller services for the reporting tasks.
|
||||
*/
|
||||
public class GetControllerServices extends AbstractNiFiCommand<ControllerServicesResult> {
|
||||
|
||||
public GetControllerServices() {
|
||||
super("get-services", ControllerServicesResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the list of controller services for the reporting tasks.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServicesResult doExecute(NiFiClient client, Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final FlowClient flowClient = client.getFlowClient();
|
||||
final ControllerServicesEntity servicesEntity = flowClient.getControllerServices();
|
||||
return new ControllerServicesResult(getResultType(properties), servicesEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.flow;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.StringResult;
|
||||
import org.apache.nifi.toolkit.cli.impl.util.JacksonUtils;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for creating a reporting task.
|
||||
*/
|
||||
public class CreateReportingTask extends AbstractNiFiCommand<StringResult> {
|
||||
|
||||
public CreateReportingTask() {
|
||||
super("create-reporting-task", StringResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Creates a reporting task from a local file.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.INPUT_SOURCE.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String inputFile = getRequiredArg(properties, CommandOption.INPUT_SOURCE);
|
||||
final URI uri = Paths.get(inputFile).toAbsolutePath().toUri();
|
||||
final String contents = IOUtils.toString(uri, StandardCharsets.UTF_8);
|
||||
|
||||
final ObjectMapper objectMapper = JacksonUtils.getObjectMapper();
|
||||
final ReportingTaskEntity deserializedTask = objectMapper.readValue(contents, ReportingTaskEntity.class);
|
||||
if (deserializedTask == null) {
|
||||
throw new IOException("Unable to deserialize reporting task from " + inputFile);
|
||||
}
|
||||
|
||||
deserializedTask.setRevision(getInitialRevisionDTO());
|
||||
|
||||
final ControllerClient controllerClient = client.getControllerClient();
|
||||
final ReportingTaskEntity createdEntity = controllerClient.createReportingTask(deserializedTask);
|
||||
|
||||
return new StringResult(String.valueOf(createdEntity.getId()), getContext().isInteractive());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.flow;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.ReportingTaskResult;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for retrieving the status of a reporting task.
|
||||
*/
|
||||
public class GetReportingTask extends AbstractNiFiCommand<ReportingTaskResult> {
|
||||
|
||||
public GetReportingTask() {
|
||||
super("get-reporting-task", ReportingTaskResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the status for a reporting task.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.RT_ID.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String rtId = getRequiredArg(properties, CommandOption.RT_ID);
|
||||
final ReportingTasksClient rtClient = client.getReportingTasksClient();
|
||||
|
||||
final ReportingTaskEntity rtEntity = rtClient.getReportingTask(rtId);
|
||||
return new ReportingTaskResult(getResultType(properties), rtEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.flow;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.FlowClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.ReportingTasksResult;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to get the list of reporting tasks.
|
||||
*/
|
||||
public class GetReportingTasks extends AbstractNiFiCommand<ReportingTasksResult> {
|
||||
|
||||
public GetReportingTasks() {
|
||||
super("get-reporting-tasks", ReportingTasksResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the list of reporting tasks.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksResult doExecute(NiFiClient client, Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final FlowClient flowClient = client.getFlowClient();
|
||||
final ReportingTasksEntity tasksEntity = flowClient.getReportingTasks();
|
||||
return new ReportingTasksResult(getResultType(properties), tasksEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,92 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.toolkit.cli.impl.command.nifi.flow;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiActivateCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command to start the reporting tasks.
|
||||
*/
|
||||
public class StartReportingTasks extends AbstractNiFiActivateCommand<ReportingTaskEntity,
|
||||
ReportingTaskRunStatusEntity> {
|
||||
|
||||
public StartReportingTasks() {
|
||||
super("start-reporting-tasks");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Starts any enabled and valid reporting tasks.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.RT_ID.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VoidResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String rtId = getArg(properties, CommandOption.RT_ID);
|
||||
final Set<ReportingTaskEntity> reportingTaskEntities = new HashSet<>();
|
||||
|
||||
if (StringUtils.isBlank(rtId)) {
|
||||
final ReportingTasksEntity reportingTasksEntity = client.getFlowClient().getReportingTasks();
|
||||
reportingTaskEntities.addAll(reportingTasksEntity.getReportingTasks());
|
||||
} else {
|
||||
reportingTaskEntities.add(client.getReportingTasksClient().getReportingTask(rtId));
|
||||
}
|
||||
|
||||
activate(client, properties, reportingTaskEntities, "RUNNING");
|
||||
|
||||
return VoidResult.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskRunStatusEntity getRunStatusEntity() {
|
||||
return new ReportingTaskRunStatusEntity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskEntity activateComponent(final NiFiClient client, final ReportingTaskEntity reportingTaskEntity,
|
||||
final ReportingTaskRunStatusEntity runStatusEntity) throws NiFiClientException, IOException {
|
||||
return client.getReportingTasksClient().activateReportingTask(reportingTaskEntity.getId(), runStatusEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDispName(final ReportingTaskEntity reportingTaskEntity) {
|
||||
return "Reporting task \"" + reportingTaskEntity.getComponent().getName() + "\" " +
|
||||
"(id: " + reportingTaskEntity.getId() + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.flow;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiActivateCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command to stop the reporting tasks.
|
||||
*/
|
||||
public class StopReportingTasks extends AbstractNiFiActivateCommand<ReportingTaskEntity, ReportingTaskRunStatusEntity> {
|
||||
|
||||
public StopReportingTasks() {
|
||||
super("stop-reporting-tasks");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Stops any running reporting tasks.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.RT_ID.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VoidResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String rtId = getArg(properties, CommandOption.RT_ID);
|
||||
final Set<ReportingTaskEntity> reportingTaskEntities = new HashSet<>();
|
||||
|
||||
if (StringUtils.isBlank(rtId)) {
|
||||
final ReportingTasksEntity reportingTasksEntity = client.getFlowClient().getReportingTasks();
|
||||
reportingTaskEntities.addAll(reportingTasksEntity.getReportingTasks());
|
||||
} else {
|
||||
reportingTaskEntities.add(client.getReportingTasksClient().getReportingTask(rtId));
|
||||
}
|
||||
|
||||
activate(client, properties, reportingTaskEntities, "STOPPED");
|
||||
|
||||
return VoidResult.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskRunStatusEntity getRunStatusEntity() {
|
||||
return new ReportingTaskRunStatusEntity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskEntity activateComponent(final NiFiClient client, final ReportingTaskEntity reportingTaskEntity,
|
||||
final ReportingTaskRunStatusEntity runStatusEntity) throws NiFiClientException, IOException {
|
||||
return client.getReportingTasksClient().activateReportingTask(reportingTaskEntity.getId(), runStatusEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDispName(final ReportingTaskEntity reportingTaskEntity) {
|
||||
return "Reporting task \"" + reportingTaskEntity.getComponent().getName() + "\" " +
|
||||
"(id: " + reportingTaskEntity.getId() + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,82 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.pg;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.StringResult;
|
||||
import org.apache.nifi.toolkit.cli.impl.util.JacksonUtils;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for creating a controller service for a given process group.
|
||||
*/
|
||||
public class PGCreateControllerService extends AbstractNiFiCommand<StringResult> {
|
||||
|
||||
public PGCreateControllerService() {
|
||||
super("pg-create-service", StringResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Creates the controller service for the given process group from the local file.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.PG_ID.createOption());
|
||||
addOption(CommandOption.INPUT_SOURCE.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String processorGroupId = getRequiredArg(properties, CommandOption.PG_ID);
|
||||
final String inputFile = getRequiredArg(properties, CommandOption.INPUT_SOURCE);
|
||||
final URI uri = Paths.get(inputFile).toAbsolutePath().toUri();
|
||||
final String contents = IOUtils.toString(uri, StandardCharsets.UTF_8);
|
||||
|
||||
final ObjectMapper objectMapper = JacksonUtils.getObjectMapper();
|
||||
final ControllerServiceEntity deserializedService = objectMapper.readValue(contents, ControllerServiceEntity.class);
|
||||
if (deserializedService == null) {
|
||||
throw new IOException("Unable to deserialize controller service version from " + inputFile);
|
||||
}
|
||||
|
||||
deserializedService.setRevision(getInitialRevisionDTO());
|
||||
|
||||
final ProcessGroupClient pgClient = client.getProcessGroupClient();
|
||||
final ControllerServiceEntity createdEntity = pgClient.createControllerService(
|
||||
processorGroupId, deserializedService);
|
||||
|
||||
return new StringResult(String.valueOf(createdEntity.getId()), getContext().isInteractive());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.policies;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.AccessPolicyAction;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.AccessPolicyResult;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to get the configuration of an access policy.
|
||||
*/
|
||||
public class GetAccessPolicy extends AbstractNiFiCommand<AccessPolicyResult> {
|
||||
|
||||
public GetAccessPolicy() {
|
||||
super("get-policy", AccessPolicyResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the configuration for an access policy.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.POLICY_RESOURCE.createOption());
|
||||
addOption(CommandOption.POLICY_ACTION.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public AccessPolicyResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, CommandException, MissingOptionException {
|
||||
final PoliciesClient policyClient = client.getPoliciesClient();
|
||||
|
||||
final String resource = getRequiredArg(properties, CommandOption.POLICY_RESOURCE);
|
||||
final AccessPolicyAction actionType = AccessPolicyAction.valueOf(
|
||||
getRequiredArg(properties, CommandOption.POLICY_ACTION).toUpperCase().trim());
|
||||
|
||||
return new AccessPolicyResult(getResultType(properties), policyClient.getAccessPolicy(
|
||||
resource, actionType.toString().toLowerCase()));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,213 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.toolkit.cli.impl.command.nifi.policies;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.AccessPolicyAction;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
|
||||
import org.apache.nifi.web.api.entity.AccessPolicyEntity;
|
||||
import org.apache.nifi.web.api.entity.TenantEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command for updating an access policy.
|
||||
*/
|
||||
public class UpdateAccessPolicy extends AbstractNiFiCommand<VoidResult> {
|
||||
|
||||
public UpdateAccessPolicy() {
|
||||
super("update-policy", VoidResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Updates the access policy for the given resource and action, or creates the policy " +
|
||||
"if it doesn't not exist. In stand-alone mode this command will not produce all of " +
|
||||
"the output seen in interactive mode unless the --verbose argument is specified.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.POLICY_RESOURCE.createOption());
|
||||
addOption(CommandOption.POLICY_ACTION.createOption());
|
||||
addOption(CommandOption.USER_NAME_LIST.createOption());
|
||||
addOption(CommandOption.USER_ID_LIST.createOption());
|
||||
addOption(CommandOption.GROUP_NAME_LIST.createOption());
|
||||
addOption(CommandOption.GROUP_ID_LIST.createOption());
|
||||
addOption(CommandOption.OVERWRITE_POLICY.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VoidResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final PoliciesClient policiesClient = client.getPoliciesClient();
|
||||
final TenantsClient tenantsClient = client.getTenantsClient();
|
||||
|
||||
final String resource = "/" + StringUtils.removeStart(getRequiredArg(properties, CommandOption.POLICY_RESOURCE), "/");
|
||||
final AccessPolicyAction actionType = AccessPolicyAction.valueOf(
|
||||
getRequiredArg(properties, CommandOption.POLICY_ACTION).toUpperCase().trim());
|
||||
|
||||
final String users = getArg(properties, CommandOption.USER_NAME_LIST);
|
||||
final String userIds = getArg(properties, CommandOption.USER_ID_LIST);
|
||||
final String groups = getArg(properties, CommandOption.GROUP_NAME_LIST);
|
||||
final String groupIds = getArg(properties, CommandOption.GROUP_ID_LIST);
|
||||
|
||||
final Set<TenantEntity> userEntities = new LinkedHashSet<>();
|
||||
|
||||
if (StringUtils.isNotBlank(users)) {
|
||||
userEntities.addAll(generateTenantEntities(users, tenantsClient.getUsers()));
|
||||
}
|
||||
|
||||
if (StringUtils.isNotBlank(userIds)) {
|
||||
userEntities.addAll(generateTenantEntities(userIds));
|
||||
}
|
||||
|
||||
final Set<TenantEntity> groupEntites = new LinkedHashSet<>();
|
||||
|
||||
if (StringUtils.isNotBlank(groups)) {
|
||||
groupEntites.addAll(generateTenantEntities(groups, tenantsClient.getUserGroups()));
|
||||
}
|
||||
|
||||
if (StringUtils.isNotBlank(groupIds)) {
|
||||
groupEntites.addAll(generateTenantEntities(groupIds));
|
||||
}
|
||||
|
||||
if (userEntities.isEmpty() && groupEntites.isEmpty()) {
|
||||
throw new CommandException("Users and groups were blank, nothing to update");
|
||||
}
|
||||
|
||||
final boolean overwrite = properties.containsKey(CommandOption.OVERWRITE_POLICY.getLongName());
|
||||
|
||||
AccessPolicyEntity policyEntity;
|
||||
try {
|
||||
policyEntity = policiesClient.getAccessPolicy(resource, actionType.toString().toLowerCase());
|
||||
} catch (NiFiClientException e) {
|
||||
policyEntity = null;
|
||||
}
|
||||
|
||||
if (policyEntity == null) {
|
||||
if (shouldPrint(properties)) {
|
||||
println("Access policy not found" +
|
||||
" for action " + actionType.toString().toLowerCase() +
|
||||
" on resource /" + StringUtils.removeStart(resource, "/"));
|
||||
}
|
||||
|
||||
final AccessPolicyDTO policyDTO = new AccessPolicyDTO();
|
||||
policyDTO.setResource(resource);
|
||||
policyDTO.setAction(actionType.toString().toLowerCase());
|
||||
policyDTO.setUsers(new LinkedHashSet<>());
|
||||
policyDTO.setUserGroups(new LinkedHashSet<>());
|
||||
|
||||
policyEntity = new AccessPolicyEntity();
|
||||
policyEntity.setComponent(policyDTO);
|
||||
policyEntity.setRevision(getInitialRevisionDTO());
|
||||
setTenant(policyEntity, userEntities, groupEntites, overwrite, properties);
|
||||
|
||||
final AccessPolicyEntity createdEntity = policiesClient.createAccessPolicy(policyEntity);
|
||||
|
||||
if (shouldPrint(properties)) {
|
||||
println("New access policy was created");
|
||||
println("id: " + createdEntity.getId());
|
||||
}
|
||||
} else if (!resource.equals(policyEntity.getComponent().getResource())) {
|
||||
if (shouldPrint(properties)) {
|
||||
println("Override the policy inherited from "
|
||||
+ policyEntity.getComponent().getResource());
|
||||
}
|
||||
|
||||
final AccessPolicyDTO policyDTO = new AccessPolicyDTO();
|
||||
policyDTO.setResource(resource);
|
||||
policyDTO.setAction(actionType.toString().toLowerCase());
|
||||
policyDTO.setUsers(policyEntity.getComponent().getUsers());
|
||||
policyDTO.setUserGroups(policyEntity.getComponent().getUserGroups());
|
||||
|
||||
policyEntity = new AccessPolicyEntity();
|
||||
policyEntity.setComponent(policyDTO);
|
||||
policyEntity.setRevision(getInitialRevisionDTO());
|
||||
setTenant(policyEntity, userEntities, groupEntites, overwrite, properties);
|
||||
|
||||
final AccessPolicyEntity createdEntity = policiesClient.createAccessPolicy(policyEntity);
|
||||
|
||||
if (shouldPrint(properties)) {
|
||||
println("Override access policy was created");
|
||||
println("id: " + createdEntity.getId());
|
||||
}
|
||||
} else {
|
||||
final String clientId = getContext().getSession().getNiFiClientID();
|
||||
policyEntity.getRevision().setClientId(clientId);
|
||||
setTenant(policyEntity, userEntities, groupEntites, overwrite, properties);
|
||||
|
||||
policiesClient.updateAccessPolicy(policyEntity);
|
||||
|
||||
if (shouldPrint(properties)) {
|
||||
println("Access policy was updated");
|
||||
println("id: " + policyEntity.getId());
|
||||
}
|
||||
}
|
||||
|
||||
return VoidResult.getInstance();
|
||||
}
|
||||
|
||||
private void setTenant(final AccessPolicyEntity policyEntity, final Set<TenantEntity> userEntities,
|
||||
final Set<TenantEntity> groupEntities, final boolean overwrite, final Properties properties) {
|
||||
if (overwrite) {
|
||||
policyEntity.getComponent().setUsers(new LinkedHashSet<>());
|
||||
policyEntity.getComponent().setUserGroups(new LinkedHashSet<>());
|
||||
}
|
||||
|
||||
final Set<TenantEntity> userSet = policyEntity.getComponent().getUsers();
|
||||
userEntities.forEach(entity -> addTenant(userSet, entity, "User", properties));
|
||||
|
||||
final Set<TenantEntity> groupSet = policyEntity.getComponent().getUserGroups();
|
||||
groupEntities.forEach(entity -> addTenant(groupSet, entity, "User group", properties));
|
||||
}
|
||||
|
||||
private void addTenant(final Set<TenantEntity> tenantSet, final TenantEntity additionalTenant,
|
||||
final String tenantType, final Properties properties) {
|
||||
final String dispTenantName = additionalTenant.getComponent() != null && StringUtils.isNotBlank(additionalTenant.getComponent().getIdentity())
|
||||
? tenantType + " \"" + additionalTenant.getComponent().getIdentity() + "\""
|
||||
: tenantType + " (id: " + additionalTenant.getId() + ")";
|
||||
|
||||
if (tenantSet.contains(additionalTenant)) {
|
||||
if (shouldPrint(properties)) {
|
||||
println(dispTenantName + " already included");
|
||||
}
|
||||
} else {
|
||||
if (shouldPrint(properties)) {
|
||||
println(dispTenantName + " added");
|
||||
}
|
||||
tenantSet.add(additionalTenant);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean shouldPrint(final Properties properties) {
|
||||
return isInteractive() || isVerbose(properties);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.templates;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TemplatesClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.TemplateResult;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to download the template file.
|
||||
*/
|
||||
public class DownloadTemplate extends AbstractNiFiCommand<TemplateResult> {
|
||||
|
||||
public DownloadTemplate() {
|
||||
super("download-template", TemplateResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Downloads the template file.";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doInitialize(final Context context) {
|
||||
addOption(CommandOption.TEMPLATE_ID.createOption());
|
||||
addOption(CommandOption.OUTPUT_FILE.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplateResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, CommandException, MissingOptionException {
|
||||
final String templateId = getRequiredArg(properties, CommandOption.TEMPLATE_ID);
|
||||
final TemplatesClient templatesClient = client.getTemplatesClient();
|
||||
|
||||
final TemplateDTO templateEntityResult = templatesClient.getTemplate(templateId);
|
||||
|
||||
// currently export doesn't use the ResultWriter concept, it always writes JSON
|
||||
// destination will be a file if outputFile is specified, otherwise it will be the output stream of the CLI
|
||||
final String outputFile;
|
||||
if (properties.containsKey(CommandOption.OUTPUT_FILE.getLongName())) {
|
||||
outputFile = properties.getProperty(CommandOption.OUTPUT_FILE.getLongName());
|
||||
} else {
|
||||
outputFile = null;
|
||||
}
|
||||
|
||||
return new TemplateResult(templateEntityResult, outputFile);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.toolkit.cli.impl.command.nifi.templates;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.TemplatesResult;
|
||||
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to get the list of templates.
|
||||
*/
|
||||
public class ListTemplates extends AbstractNiFiCommand<TemplatesResult> {
|
||||
|
||||
public ListTemplates() {
|
||||
super("list-templates", TemplatesResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the list of templates";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final TemplatesEntity templatesEntity = client.getFlowClient().getTemplates();
|
||||
return new TemplatesResult(getResultType(properties), templatesEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.templates;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.persistence.TemplateDeserializer;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.StringResult;
|
||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for uploading a template file.
|
||||
*/
|
||||
public class UploadTemplate extends AbstractNiFiCommand<StringResult> {
|
||||
|
||||
public UploadTemplate() {
|
||||
super("upload-template", StringResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Uploads a local template file.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.PG_ID.createOption());
|
||||
addOption(CommandOption.INPUT_SOURCE.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final String processGroupId = getRequiredArg(properties, CommandOption.PG_ID);
|
||||
final String inputFile = getRequiredArg(properties, CommandOption.INPUT_SOURCE);
|
||||
|
||||
final FileInputStream file = new FileInputStream(Paths.get(inputFile).toAbsolutePath().toFile());
|
||||
|
||||
final ProcessGroupClient pgClient = client.getProcessGroupClient();
|
||||
final TemplateEntity createdEntity = pgClient.uploadTemplate(
|
||||
processGroupId,
|
||||
TemplateDeserializer.deserialize(file));
|
||||
|
||||
return new StringResult(String.valueOf(createdEntity.getTemplate().getId()), getContext().isInteractive());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.tenants;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.StringResult;
|
||||
import org.apache.nifi.web.api.dto.UserDTO;
|
||||
import org.apache.nifi.web.api.entity.UserEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command for creating a user.
|
||||
*/
|
||||
public class CreateUser extends AbstractNiFiCommand<StringResult> {
|
||||
|
||||
public CreateUser() {
|
||||
super("create-user", StringResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Creates new user.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.USER_NAME.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException {
|
||||
|
||||
final String userId = getRequiredArg(properties, CommandOption.USER_NAME);
|
||||
|
||||
final UserDTO userDTO = new UserDTO();
|
||||
userDTO.setIdentity(userId);
|
||||
|
||||
final UserEntity userEntity = new UserEntity();
|
||||
userEntity.setComponent(userDTO);
|
||||
userEntity.setRevision(getInitialRevisionDTO());
|
||||
|
||||
final UserEntity createdEntity = client.getTenantsClient().createUser(userEntity);
|
||||
return new StringResult(createdEntity.getId(), getContext().isInteractive());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.tenants;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.StringResult;
|
||||
import org.apache.nifi.web.api.dto.UserGroupDTO;
|
||||
import org.apache.nifi.web.api.entity.TenantEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command for creating a user group.
|
||||
*/
|
||||
public class CreateUserGroup extends AbstractNiFiCommand<StringResult> {
|
||||
|
||||
public CreateUserGroup() {
|
||||
super("create-user-group", StringResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Creates new user group.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.UG_NAME.createOption());
|
||||
addOption(CommandOption.USER_NAME_LIST.createOption());
|
||||
addOption(CommandOption.USER_ID_LIST.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
|
||||
final String userGroupId = getRequiredArg(properties, CommandOption.UG_NAME);
|
||||
final String users = getArg(properties, CommandOption.USER_NAME_LIST);
|
||||
final String userIds = getArg(properties, CommandOption.USER_ID_LIST);
|
||||
|
||||
final UserGroupDTO userGroupDTO = new UserGroupDTO();
|
||||
userGroupDTO.setIdentity(userGroupId);
|
||||
|
||||
final Set<TenantEntity> tenantEntities = new HashSet<>();
|
||||
|
||||
if (StringUtils.isNotBlank(users)) {
|
||||
tenantEntities.addAll(
|
||||
generateTenantEntities(users, client.getTenantsClient().getUsers()));
|
||||
}
|
||||
|
||||
if (StringUtils.isNotBlank(userIds)) {
|
||||
tenantEntities.addAll(generateTenantEntities(userIds));
|
||||
}
|
||||
|
||||
userGroupDTO.setUsers(tenantEntities);
|
||||
|
||||
final UserGroupEntity userGroupEntity = new UserGroupEntity();
|
||||
userGroupEntity.setComponent(userGroupDTO);
|
||||
userGroupEntity.setRevision(getInitialRevisionDTO());
|
||||
|
||||
final UserGroupEntity createdEntity = client.getTenantsClient().createUserGroup(userGroupEntity);
|
||||
return new StringResult(createdEntity.getId(), getContext().isInteractive());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.tenants;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.UserGroupsResult;
|
||||
import org.apache.nifi.web.api.entity.UserGroupsEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to get the list of user groups.
|
||||
*/
|
||||
public class ListUserGroups extends AbstractNiFiCommand<UserGroupsResult> {
|
||||
|
||||
public ListUserGroups() {
|
||||
super("list-user-groups", UserGroupsResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the list of user group.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserGroupsResult doExecute(NiFiClient client, Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final TenantsClient tenantsClient = client.getTenantsClient();
|
||||
final UserGroupsEntity userGroupsEntity = tenantsClient.getUserGroups();
|
||||
return new UserGroupsResult(getResultType(properties), userGroupsEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.tenants;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.UsersResult;
|
||||
import org.apache.nifi.web.api.entity.UsersEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Command to get the list of users.
|
||||
*/
|
||||
public class ListUsers extends AbstractNiFiCommand<UsersResult> {
|
||||
|
||||
public ListUsers() {
|
||||
super("list-users", UsersResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Retrieves the list of user.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public UsersResult doExecute(NiFiClient client, Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final TenantsClient tenantsClient = client.getTenantsClient();
|
||||
final UsersEntity usersEntity = tenantsClient.getUsers();
|
||||
return new UsersResult(getResultType(properties), usersEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.command.nifi.tenants;
|
||||
|
||||
import org.apache.commons.cli.MissingOptionException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.toolkit.cli.api.CommandException;
|
||||
import org.apache.nifi.toolkit.cli.api.Context;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
|
||||
import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
|
||||
import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.VoidResult;
|
||||
import org.apache.nifi.web.api.entity.TenantEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Command for updating a user group.
|
||||
*/
|
||||
public class UpdateUserGroup extends AbstractNiFiCommand<VoidResult> {
|
||||
|
||||
public UpdateUserGroup() {
|
||||
super("update-user-group", VoidResult.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Updates a user group.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doInitialize(final Context context) {
|
||||
addOption(CommandOption.UG_NAME.createOption());
|
||||
addOption(CommandOption.UG_ID.createOption());
|
||||
addOption(CommandOption.USER_NAME_LIST.createOption());
|
||||
addOption(CommandOption.USER_ID_LIST.createOption());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VoidResult doExecute(final NiFiClient client, final Properties properties)
|
||||
throws NiFiClientException, IOException, MissingOptionException, CommandException {
|
||||
final TenantsClient tenantsClient = client.getTenantsClient();
|
||||
|
||||
final String group = getArg(properties, CommandOption.UG_NAME);
|
||||
final String groupId = getArg(properties, CommandOption.UG_ID);
|
||||
|
||||
if ((StringUtils.isBlank(group) && StringUtils.isBlank(groupId))
|
||||
|| (StringUtils.isNotBlank(group) && StringUtils.isNotBlank(groupId))) {
|
||||
throw new CommandException("Specify either \"" + CommandOption.UG_NAME.getLongName()
|
||||
+ "\" or \"" + CommandOption.UG_ID.getLongName() + "\" (not both)");
|
||||
}
|
||||
|
||||
UserGroupEntity existingGroup;
|
||||
|
||||
if (StringUtils.isNotBlank(group)) {
|
||||
final Optional<UserGroupEntity> existingGroupEntity = tenantsClient.getUserGroups().getUserGroups().stream()
|
||||
.filter(userGroupEntity -> group.equals(userGroupEntity.getComponent().getIdentity()))
|
||||
.findAny();
|
||||
|
||||
if (!existingGroupEntity.isPresent()) {
|
||||
throw new CommandException("User group does not exist for identity \"" + group + "\"");
|
||||
}
|
||||
|
||||
existingGroup = existingGroupEntity.get();
|
||||
} else {
|
||||
existingGroup = tenantsClient.getUserGroup(groupId);
|
||||
}
|
||||
|
||||
final String users = getArg(properties, CommandOption.USER_NAME_LIST);
|
||||
final String userIds = getArg(properties, CommandOption.USER_ID_LIST);
|
||||
|
||||
final Set<TenantEntity> tenantEntities = new HashSet<>();
|
||||
|
||||
if (StringUtils.isNotBlank(users)) {
|
||||
tenantEntities.addAll(
|
||||
generateTenantEntities(users, client.getTenantsClient().getUsers()));
|
||||
}
|
||||
|
||||
if (StringUtils.isNotBlank(userIds)) {
|
||||
tenantEntities.addAll(generateTenantEntities(userIds));
|
||||
}
|
||||
|
||||
existingGroup.getComponent().setUsers(tenantEntities);
|
||||
|
||||
final String clientId = getContext().getSession().getNiFiClientID();
|
||||
existingGroup.getRevision().setClientId(clientId);
|
||||
|
||||
tenantsClient.updateUserGroup(existingGroup);
|
||||
return VoidResult.getInstance();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
|
||||
import org.apache.nifi.web.api.entity.AccessPolicyEntity;
|
||||
import org.apache.nifi.web.api.entity.TenantEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class AccessPolicyResult extends AbstractWritableResult<AccessPolicyEntity> {
|
||||
|
||||
private final AccessPolicyEntity accessPolicyEntity;
|
||||
|
||||
public AccessPolicyResult(ResultType resultType, AccessPolicyEntity accessPolicyEntity) {
|
||||
super(resultType);
|
||||
this.accessPolicyEntity = accessPolicyEntity;
|
||||
Validate.notNull(accessPolicyEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AccessPolicyEntity getResult() {
|
||||
return accessPolicyEntity;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(PrintStream output) throws IOException {
|
||||
final AccessPolicyDTO accessPolicyDTO = accessPolicyEntity.getComponent();
|
||||
|
||||
output.printf("Resource: %s\nAction : %s\nUsers : %s\nGroups : %s\n",
|
||||
accessPolicyDTO.getResource(),
|
||||
accessPolicyDTO.getAction(),
|
||||
joinTenantIdentity(accessPolicyDTO.getUsers()),
|
||||
joinTenantIdentity(accessPolicyDTO.getUserGroups())
|
||||
);
|
||||
}
|
||||
|
||||
private String joinTenantIdentity(Set<TenantEntity> entities) {
|
||||
return entities.stream()
|
||||
.map(e -> e.getComponent() != null ? e.getComponent().getIdentity() : e.getId())
|
||||
.collect(Collectors.joining(", "));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.web.api.dto.BundleDTO;
|
||||
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
||||
public class ControllerServiceResult extends AbstractWritableResult<ControllerServiceEntity> {
|
||||
|
||||
private final ControllerServiceEntity controllerServiceEntity;
|
||||
|
||||
public ControllerServiceResult(ResultType resultType, ControllerServiceEntity controllerServiceEntity) {
|
||||
super(resultType);
|
||||
this.controllerServiceEntity = controllerServiceEntity;
|
||||
Validate.notNull(controllerServiceEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceEntity getResult() {
|
||||
return controllerServiceEntity;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(PrintStream output) throws IOException {
|
||||
final ControllerServiceDTO controllerServiceDTO = controllerServiceEntity.getComponent();
|
||||
|
||||
final BundleDTO bundle = controllerServiceDTO.getBundle();
|
||||
output.printf("Name : %s\nID : %s\nType : %s\nBundle: %s - %s %s\nState : %s\n",
|
||||
controllerServiceDTO.getName(), controllerServiceDTO.getId(), controllerServiceDTO.getType(),
|
||||
bundle.getGroup(), bundle.getArtifact(), bundle.getVersion(), controllerServiceDTO.getState());
|
||||
}
|
||||
}
|
|
@ -62,12 +62,13 @@ public class ControllerServicesResult extends AbstractWritableResult<ControllerS
|
|||
final Table table = new Table.Builder()
|
||||
.column("#", 3, 3, false)
|
||||
.column("Name", 5, 40, false)
|
||||
.column("ID", 36, 36, false)
|
||||
.column("State", 5, 40, false)
|
||||
.build();
|
||||
|
||||
for (int i=0; i < serviceDTOS.size(); i++) {
|
||||
final ControllerServiceDTO serviceDTO = serviceDTOS.get(i);
|
||||
table.addRow(String.valueOf(i+1), serviceDTO.getName(), serviceDTO.getState());
|
||||
table.addRow(String.valueOf(i+1), serviceDTO.getName(), serviceDTO.getId(), serviceDTO.getState());
|
||||
}
|
||||
|
||||
final TableWriter tableWriter = new DynamicTableWriter();
|
||||
|
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.web.api.dto.BundleDTO;
|
||||
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
||||
public class ReportingTaskResult extends AbstractWritableResult<ReportingTaskEntity> {
|
||||
|
||||
private final ReportingTaskEntity reportingTaskEntity;
|
||||
|
||||
public ReportingTaskResult(final ResultType resultType, final ReportingTaskEntity reportingTaskEntity) {
|
||||
super(resultType);
|
||||
this.reportingTaskEntity = reportingTaskEntity;
|
||||
Validate.notNull(reportingTaskEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTaskEntity getResult() {
|
||||
return reportingTaskEntity;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(final PrintStream output) throws IOException {
|
||||
final ReportingTaskDTO reportingTaskDTO = reportingTaskEntity.getComponent();
|
||||
|
||||
final BundleDTO bundle = reportingTaskDTO.getBundle();
|
||||
output.printf("Name : %s\nID : %s\nType : %s\nBundle: %s - %s %s\nState : %s\n",
|
||||
reportingTaskDTO.getName(), reportingTaskDTO.getId(), reportingTaskDTO.getType(),
|
||||
bundle.getGroup(), bundle.getArtifact(), bundle.getVersion(), reportingTaskDTO.getState());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.DynamicTableWriter;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.Table;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.TableWriter;
|
||||
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Result for ReportingTasksEntity.
|
||||
*/
|
||||
public class ReportingTasksResult extends AbstractWritableResult<ReportingTasksEntity> {
|
||||
|
||||
private final ReportingTasksEntity reportingTasksEntity;
|
||||
|
||||
public ReportingTasksResult(final ResultType resultType, final ReportingTasksEntity reportingTasksEntity) {
|
||||
super(resultType);
|
||||
this.reportingTasksEntity = reportingTasksEntity;
|
||||
Validate.notNull(this.reportingTasksEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(final PrintStream output) throws IOException {
|
||||
final Set<ReportingTaskEntity> tasksEntities = reportingTasksEntity.getReportingTasks();
|
||||
if (tasksEntities == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final List<ReportingTaskDTO> taskDTOS = tasksEntities.stream()
|
||||
.map(ReportingTaskEntity::getComponent)
|
||||
.sorted(Comparator.comparing(ReportingTaskDTO::getName))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
final Table table = new Table.Builder()
|
||||
.column("#", 3, 3, false)
|
||||
.column("Name", 5, 40, true)
|
||||
.column("ID", 36, 36, false)
|
||||
.column("Type", 5, 40, true)
|
||||
.column("Run Status", 10, 20, false)
|
||||
.build();
|
||||
|
||||
for (int i = 0; i < taskDTOS.size(); i++) {
|
||||
final ReportingTaskDTO taskDTO = taskDTOS.get(i);
|
||||
final String[] typeSplit = taskDTO.getType().split("\\.", -1);
|
||||
table.addRow(
|
||||
String.valueOf(i + 1),
|
||||
taskDTO.getName(),
|
||||
taskDTO.getId(),
|
||||
typeSplit[typeSplit.length - 1],
|
||||
taskDTO.getState()
|
||||
);
|
||||
}
|
||||
|
||||
final TableWriter tableWriter = new DynamicTableWriter();
|
||||
tableWriter.write(table, output);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReportingTasksEntity getResult() {
|
||||
return reportingTasksEntity;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.persistence.TemplateSerializer;
|
||||
import org.apache.nifi.toolkit.cli.api.WritableResult;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.io.PrintStream;
|
||||
|
||||
public class TemplateResult implements WritableResult<TemplateDTO> {
|
||||
|
||||
private final TemplateDTO templateDTO;
|
||||
|
||||
private final String exportFileName;
|
||||
|
||||
public TemplateResult(final TemplateDTO templateDTO, final String exportFileName) {
|
||||
this.templateDTO = templateDTO;
|
||||
this.exportFileName = exportFileName;
|
||||
Validate.notNull(this.templateDTO);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplateDTO getResult() {
|
||||
return templateDTO;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(final PrintStream output) throws IOException {
|
||||
final byte[] serializedTemplate = TemplateSerializer.serialize(templateDTO);
|
||||
if (exportFileName != null) {
|
||||
try (final OutputStream resultOut = new FileOutputStream(exportFileName)) {
|
||||
resultOut.write(serializedTemplate);
|
||||
}
|
||||
} else {
|
||||
output.write(serializedTemplate);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.DynamicTableWriter;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.Table;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.TableWriter;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Result for TemplatesEntity
|
||||
*/
|
||||
public class TemplatesResult extends AbstractWritableResult<TemplatesEntity> {
|
||||
|
||||
private final TemplatesEntity templatesEntity;
|
||||
|
||||
public TemplatesResult(final ResultType resultType, final TemplatesEntity templatesEntity) {
|
||||
super(resultType);
|
||||
this.templatesEntity = templatesEntity;
|
||||
Validate.notNull(this.templatesEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(final PrintStream output) throws IOException {
|
||||
final Collection<TemplateEntity> templateEntities = templatesEntity.getTemplates();
|
||||
if (templateEntities == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final List<TemplateDTO> templateDTOS = templateEntities.stream()
|
||||
.map(TemplateEntity::getTemplate)
|
||||
.sorted(Comparator.comparing(TemplateDTO::getGroupId))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
final Table table = new Table.Builder()
|
||||
.column("#", 1, 4, false)
|
||||
.column("Name", 5, 40, false)
|
||||
.column("ID", 36, 36, false)
|
||||
.column("Group ID", 36, 36, false)
|
||||
.build();
|
||||
|
||||
for (int i = 0; i < templateDTOS.size(); i++) {
|
||||
final TemplateDTO templateDTO = templateDTOS.get(i);
|
||||
table.addRow(
|
||||
String.valueOf(i + 1),
|
||||
templateDTO.getName(),
|
||||
templateDTO.getId(),
|
||||
templateDTO.getGroupId()
|
||||
);
|
||||
}
|
||||
|
||||
final TableWriter tableWriter = new DynamicTableWriter();
|
||||
tableWriter.write(table, output);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TemplatesEntity getResult() {
|
||||
return templatesEntity;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.DynamicTableWriter;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.Table;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.TableWriter;
|
||||
import org.apache.nifi.web.api.dto.UserGroupDTO;
|
||||
import org.apache.nifi.web.api.entity.UserGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.UserGroupsEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Result for UserGroupsEntity.
|
||||
*/
|
||||
public class UserGroupsResult extends AbstractWritableResult<UserGroupsEntity> {
|
||||
|
||||
private final UserGroupsEntity userGroupsEntity;
|
||||
|
||||
public UserGroupsResult(final ResultType resultType, final UserGroupsEntity userGroupsEntity) {
|
||||
super(resultType);
|
||||
this.userGroupsEntity = userGroupsEntity;
|
||||
Validate.notNull(this.userGroupsEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(final PrintStream output) throws IOException {
|
||||
final Collection<UserGroupEntity> userGroupEntities = userGroupsEntity.getUserGroups();
|
||||
if (userGroupEntities == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final List<UserGroupDTO> userGroupDTOS = userGroupEntities.stream()
|
||||
.map(s -> s.getComponent())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Collections.sort(userGroupDTOS, Comparator.comparing(UserGroupDTO::getIdentity));
|
||||
|
||||
final Table table = new Table.Builder()
|
||||
.column("#", 3, 3, false)
|
||||
.column("Name", 5, 40, false)
|
||||
.column("ID", 36, 36, false)
|
||||
.column("Members", 20, 40, true)
|
||||
.build();
|
||||
|
||||
for (int i = 0; i < userGroupDTOS.size(); i++) {
|
||||
final UserGroupDTO userGroupDTO = userGroupDTOS.get(i);
|
||||
table.addRow(
|
||||
String.valueOf(i + 1),
|
||||
userGroupDTO.getIdentity(),
|
||||
userGroupDTO.getId(),
|
||||
userGroupDTO.getUsers().stream().map(u -> u.getComponent().getIdentity())
|
||||
.collect(Collectors.joining(", "))
|
||||
);
|
||||
}
|
||||
|
||||
final TableWriter tableWriter = new DynamicTableWriter();
|
||||
tableWriter.write(table, output);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UserGroupsEntity getResult() {
|
||||
return userGroupsEntity;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.toolkit.cli.impl.result;
|
||||
|
||||
import org.apache.commons.lang3.Validate;
|
||||
import org.apache.nifi.toolkit.cli.api.ResultType;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.DynamicTableWriter;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.Table;
|
||||
import org.apache.nifi.toolkit.cli.impl.result.writer.TableWriter;
|
||||
import org.apache.nifi.web.api.dto.UserDTO;
|
||||
import org.apache.nifi.web.api.entity.UserEntity;
|
||||
import org.apache.nifi.web.api.entity.UsersEntity;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Result for UsersEntity.
|
||||
*/
|
||||
public class UsersResult extends AbstractWritableResult<UsersEntity> {
|
||||
|
||||
private final UsersEntity usersEntity;
|
||||
|
||||
public UsersResult(final ResultType resultType, final UsersEntity usersEntity) {
|
||||
super(resultType);
|
||||
this.usersEntity = usersEntity;
|
||||
Validate.notNull(this.usersEntity);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeSimpleResult(final PrintStream output) throws IOException {
|
||||
final Collection<UserEntity> userEntities = usersEntity.getUsers();
|
||||
if (userEntities == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final List<UserDTO> userDTOS = userEntities.stream()
|
||||
.map(s -> s.getComponent())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Collections.sort(userDTOS, Comparator.comparing(UserDTO::getIdentity));
|
||||
|
||||
final Table table = new Table.Builder()
|
||||
.column("#", 3, 3, false)
|
||||
.column("Name", 5, 40, false)
|
||||
.column("ID", 36, 36, false)
|
||||
.column("Member of", 20, 40, true)
|
||||
.build();
|
||||
|
||||
for (int i = 0; i < userDTOS.size(); i++) {
|
||||
final UserDTO userDTO = userDTOS.get(i);
|
||||
table.addRow(
|
||||
String.valueOf(i + 1),
|
||||
userDTO.getIdentity(),
|
||||
userDTO.getId(),
|
||||
userDTO.getUserGroups().stream().map(u -> u.getComponent().getIdentity())
|
||||
.collect(Collectors.joining(", "))
|
||||
);
|
||||
}
|
||||
|
||||
final TableWriter tableWriter = new DynamicTableWriter();
|
||||
tableWriter.write(table, output);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UsersEntity getResult() {
|
||||
return usersEntity;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue