-
Notifications
You must be signed in to change notification settings - Fork 2.9k
NIFI-15919: Ensure that we only sync Connectors with Connector Config… #11231
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,50 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.nifi.components.connector; | ||
|
|
||
| /** | ||
| * Indicates whether retrieval of {@link ConnectorNode} instances from the | ||
| * {@link ConnectorRepository} should consult the configured | ||
| * {@link ConnectorConfigurationProvider} to refresh the local state from the | ||
| * external store, or whether the in-memory copy should be returned as-is. | ||
| * | ||
| * <p>Choosing {@link #SYNC_WITH_PROVIDER} causes the repository to call into | ||
| * the configuration provider (which may perform network I/O, secret lookups, | ||
| * and other potentially expensive or failure-prone operations) before returning | ||
| * the connector. This is appropriate for user-facing reads that must reflect | ||
| * the latest external state, such as REST API responses. Callers operating on | ||
| * critical paths that must not block on or fail because of the external store | ||
| * (for example, periodic flow serialization) should use {@link #LOCAL_ONLY} | ||
| * instead.</p> | ||
| */ | ||
| public enum ConnectorSyncMode { | ||
|
|
||
| /** | ||
| * Consult the {@link ConnectorConfigurationProvider} (when configured) to | ||
| * refresh the local connector state from the external store before | ||
| * returning. This may perform network I/O and may throw if the external | ||
| * store is unavailable. | ||
| */ | ||
| SYNC_WITH_PROVIDER, | ||
|
|
||
| /** | ||
| * Return the in-memory connector state without consulting the | ||
| * {@link ConnectorConfigurationProvider}. No external I/O is performed. | ||
| */ | ||
| LOCAL_ONLY | ||
|
mcgilman marked this conversation as resolved.
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -459,19 +459,23 @@ private void purgeConnectorAndAwait(final ConnectorNode connector) { | |
| } | ||
|
|
||
| @Override | ||
| public ConnectorNode getConnector(final String identifier) { | ||
| public ConnectorNode getConnector(final String identifier, final ConnectorSyncMode syncMode) { | ||
| Objects.requireNonNull(syncMode, "syncMode is required"); | ||
| final ConnectorNode connector = connectors.get(identifier); | ||
| if (connector != null) { | ||
| if (connector != null && syncMode == ConnectorSyncMode.SYNC_WITH_PROVIDER) { | ||
| syncFromProvider(connector); | ||
| } | ||
| return connector; | ||
| } | ||
|
|
||
| @Override | ||
| public List<ConnectorNode> getConnectors() { | ||
| public List<ConnectorNode> getConnectors(final ConnectorSyncMode syncMode) { | ||
| Objects.requireNonNull(syncMode, "syncMode is required"); | ||
| final List<ConnectorNode> connectorList = List.copyOf(connectors.values()); | ||
| for (final ConnectorNode connector : connectorList) { | ||
| syncFromProvider(connector); | ||
| if (syncMode == ConnectorSyncMode.SYNC_WITH_PROVIDER) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This assumes that a
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good call — added |
||
| for (final ConnectorNode connector : connectorList) { | ||
| syncFromProvider(connector); | ||
| } | ||
| } | ||
| return connectorList; | ||
| } | ||
|
|
@@ -665,7 +669,9 @@ private void collectReferencedAssetIds(final FrameworkFlowContext flowContext, f | |
| @Override | ||
| public void updateConnector(final ConnectorNode connector, final String name) { | ||
| if (configurationProvider != null) { | ||
| final ConnectorWorkingConfiguration workingConfiguration = buildWorkingConfiguration(connector); | ||
| // Load the latest provider state so that other in-flight working changes are not overwritten by a rename. | ||
| final Optional<ConnectorWorkingConfiguration> externalConfig = configurationProvider.load(connector.getIdentifier()); | ||
| final ConnectorWorkingConfiguration workingConfiguration = externalConfig.orElseGet(() -> buildWorkingConfiguration(connector)); | ||
| workingConfiguration.setName(name); | ||
| configurationProvider.save(connector.getIdentifier(), workingConfiguration); | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.